SlideShare a Scribd company logo
Building Streaming Applications with
Apache Apex
Chinmay Kolhatkar, Committer @ApacheApex, Engineer @DataTorrent
Thomas Weise, PMC Chair @ApacheApex, Architect @DataTorrent
Nov 15th
2016
Agenda
2
• Application Development Model
• Creating Apex Application - Project Structure
• Apex APIs
• Configuration Example
• Operator APIs
• Overview of Operator Library
• Frequently used Connectors
• Stateful Transformation & Windowing
• Scalability - Partitioning
• End-to-end Exactly Once
Application Development Model
3
▪Stream is a sequence of data tuples
▪Operator takes one or more input streams, performs computations & emits one or more output streams
• Each Operator is YOUR custom business logic in java, or built-in operator from our open source library
• Operator has many instances that run in parallel and each instance is single-threaded
▪Directed Acyclic Graph (DAG) is made up of operators and streams
Directed Acyclic Graph (DAG)
Filtered
Stream
Output
Stream
Tuple Tuple
FilteredStream
Enriched
Stream
Enriched
Stream
er
Operator
er
Operator
er
Operator
er
Operator
er
Operator
er
Operator
Creating Apex Application Project
4
chinmay@chinmay-VirtualBox:~/src$ mvn archetype:generate -DarchetypeGroupId=org.apache.apex
-DarchetypeArtifactId=apex-app-archetype -DarchetypeVersion=LATEST -DgroupId=com.example
-Dpackage=com.example.myapexapp -DartifactId=myapexapp -Dversion=1.0-SNAPSHOT
…
…
...
Confirm properties configuration:
groupId: com.example
artifactId: myapexapp
version: 1.0-SNAPSHOT
package: com.example.myapexapp
archetypeVersion: LATEST
Y: : Y
…
…
...
[INFO] project created from Archetype in dir: /media/sf_workspace/src/myapexapp
[INFO] ------------------------------------------------------------------------
[INFO] BUILD SUCCESS
[INFO] ------------------------------------------------------------------------
[INFO] Total time: 13.141 s
[INFO] Finished at: 2016-11-15T14:06:56+05:30
[INFO] Final Memory: 18M/216M
[INFO] ------------------------------------------------------------------------
chinmay@chinmay-VirtualBox:~/src$
https://www.youtube.com/watch?v=z-eeh-tjQrc
Apex Application Project Structure
5
• pom.xml
• Defines project structure and
dependencies
• Application.java
• Defines the DAG
• RandomNumberGenerator.java
• Sample Operator
• properties.xml
• Contains operator and application
properties and attributes
• ApplicationTest.java
• Sample test to test application in local
mode
Apex APIs: Compositional (Low level)
6
Input Parser Counter Output
CountsWordsLines
Kafka Database
Filter
Filtered
Apex APIs: Declarative (High Level)
7
File
Input
Parser
Word
Counter
Console
Output
CountsWordsLines
Folder StdOut
StreamFactory.fromFolder("/tmp")
.flatMap(input -> Arrays.asList(input.split(" ")), name("Words"))
.window(new WindowOption.GlobalWindow(),
new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(1))
.countByKey(input -> new Tuple.PlainTuple<>(new KeyValPair<>(input, 1L)), name("countByKey"))
.map(input -> input.getValue(), name("Counts"))
.print(name("Console"))
.populateDag(dag);
Apex APIs: SQL
8
Kafka
Input
CSV
Parser
Filter CSV
Formattter
FilteredWordsLines
Kafka File
Project
Projected
Line
Writer
Formatted
SQLExecEnvironment.getEnvironment()
.registerTable("ORDERS",
new KafkaEndpoint(conf.get("broker"), conf.get("topic"),
new CSVMessageFormat(conf.get("schemaInDef"))))
.registerTable("SALES",
new FileEndpoint(conf.get("destFolder"), conf.get("destFileName"),
new CSVMessageFormat(conf.get("schemaOutDef"))))
.registerFunction("APEXCONCAT", this.getClass(), "apex_concat_str")
.executeSQL(dag,
"INSERT INTO SALES " +
"SELECT STREAM ROWTIME, FLOOR(ROWTIME TO DAY), APEXCONCAT('OILPAINT', SUBSTRING(PRODUCT, 6, 7) " +
"FROM ORDERS WHERE ID > 3 AND PRODUCT LIKE 'paint%'");
Apex APIs: Beam
9
• Apex Runner of Beam is available!!
• Build once run-anywhere model
• Beam Streaming applications can be run on apex runner:
public static void main(String[] args) {
Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
// Run with Apex runner
options.setRunner(ApexRunner.class);
Pipeline p = Pipeline.create(options);
p.apply("ReadLines", TextIO.Read.from(options.getInput()))
.apply(new CountWords())
.apply(MapElements.via(new FormatAsTextFn()))
.apply("WriteCounts", TextIO.Write.to(options.getOutput()));
.run().waitUntilFinish();
}
Apex APIs: SAMOA
10
• Build once run-anywhere model for online machine learning algorithms
• Any machine learning algorithm present in SAMOA can be run directly on Apex.
• Uses Apex Iteration Support
• Following example does classification of input data from HDFS using VHT algorithm on
Apex:
$ bin/samoa apex ../SAMOA-Apex-0.4.0-incubating-SNAPSHOT.jar "PrequentialEvaluation
-d /tmp/dump.csv
-l (classifiers.trees.VerticalHoeffdingTree -p 1)
-s (org.apache.samoa.streams.ArffFileStream
-s HDFSFileStreamSource
-f /tmp/user/input/covtypeNorm.arff)"
Configuration (properties.xml)
11
Input Parser Counter Output
CountsWordsLines
Kafka Database
Filter
Filtered
Streaming Window
Processing Time Window
12
• Finite time sliced windows based on processing (event arrival) time
• Used for bookkeeping of streaming application
• Derived Windows are: Checkpoint Windows, Committed Windows
Operator APIs
13
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()
Overview of Operator Library (Malhar)
14
RDBMS
• JDBC
• MySQL
• Oracle
• MemSQL
NoSQL
• Cassandra, HBase
• Aerospike, Accumulo
• Couchbase/ CouchDB
• Redis, MongoDB
• Geode
Messaging
• Kafka
• JMS (ActiveMQ etc.)
• Kinesis, SQS
• Flume, NiFi
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
• MQTT
• SMTP
Other
• Elastic Search
• Script (JavaScript, Python, R)
• Solr
• Twitter
Frequently used Connectors
Kafka Input
15
KafkaSinglePortInputOperator KafkaSinglePortByteArrayInputOperator
Library malhar-contrib malhar-kafka
Kafka Consumer 0.8 0.9
Emit Type byte[] byte[]
Fault-Tolerance At Least Once, Exactly Once At Least Once, Exactly Once
Scalability Static and Dynamic (with Kafka
metadata)
Static and Dynamic (with Kafka metadata)
Multi-Cluster/Topic Yes Yes
Idempotent Yes Yes
Partition Strategy 1:1, 1:M 1:1, 1:M
Frequently used Connectors
Kafka Output
16
KafkaSinglePortOutputOperator KafkaSinglePortExactlyOnceOutputOperator
Library malhar-contrib malhar-kafka
Kafka Producer 0.8 0.9
Fault-Tolerance At Least Once At Least Once, Exactly Once
Scalability Static and Dynamic (with Kafka
metadata)
Static and Dynamic, Automatic Partitioning
based on Kafka metadata
Multi-Cluster/Topic Yes Yes
Idempotent Yes Yes
Partition Strategy 1:1, 1:M 1:1, 1:M
Frequently used Connectors
File Input
17
• AbstractFileInputOperator
• Used to read a file from source and
emit the content of the file to
downstream operator
• Operator is idempotent
• Supports Partitioning
• Few Concrete Impl
• FileLineInputOperator
• AvroFileInputOperator
• ParquetFilePOJOReader
• https://www.datatorrent.com/blog/f
ault-tolerant-file-processing/
Frequently used Connectors
File Output
18
• AbstractFileOutputOperator
• Writes data to a file
• Supports Partitions
• Exactly-once results
• Upstream operators should be
idempotent
• Few Concrete Impl
• StringFileOutputOperator
• https://www.datatorrent.com/blog/f
ault-tolerant-file-processing/
Windowing Support
19
• Event-time Windows
• Computation based on event-time present in the tuple
• Types of event-time windows supported:
• Global : Single event-time window throughout the lifecycle of application
• Timed : Tuple is assigned to single, non-overlapping, fixed width windows immediately
followed by next window
• Sliding Time : Tuple is can be assigned to multiple, overlapping fixed width windows.
• Session : Tuple is assigned to single, variable width windows with a predefined min gap
Stateful Windowed Processing
20
• WindowedOperator from malhar-library
• Used to process data based on Event time as contrary to ingression time
• Supports windowing semantics of Apache Beam model
• Supported features:
• Watermarks
• Allowed Lateness
• Accumulation
• Accumulation Modes: Accumulating, Discarding, Accumulating & Retracting
• Triggers
• Storage
• In memory based
• Managed State based
Stateful Windowed Processing
Compositional API
21
@Override
public void populateDAG(DAG dag, Configuration configuration)
{
WordGenerator inputOperator = new WordGenerator();
KeyedWindowedOperatorImpl windowedOperator = new KeyedWindowedOperatorImpl();
Accumulation<Long, MutableLong, Long> sum = new SumAccumulation();
windowedOperator.setAccumulation(sum);
windowedOperator.setDataStorage(new InMemoryWindowedKeyedStorage<String, MutableLong>());
windowedOperator.setRetractionStorage(new InMemoryWindowedKeyedStorage<String, Long>());
windowedOperator.setWindowStateStorage(new InMemoryWindowedStorage<WindowState>());
windowedOperator.setWindowOption(new WindowOption.TimeWindows(Duration.standardMinutes(1)));
windowedOperator.setTriggerOption(TriggerOption.AtWatermark()
.withEarlyFiringsAtEvery(Duration.millis(1000))
.accumulatingAndRetractingFiredPanes());
windowedOperator.setAllowedLateness(Duration.millis(14000));
ConsoleOutputOperator outputOperator = new ConsoleOutputOperator();
dag.addOperator( "inputOperator", inputOperator);
dag.addOperator( "windowedOperator", windowedOperator);
dag.addOperator( "outputOperator", outputOperator);
dag.addStream( "input_windowed", inputOperator. output, windowedOperator.input);
dag.addStream( "windowed_output", windowedOperator.output, outputOperator. input);
}
Stateful Windowed Processing
Declarative API
22
StreamFactory.fromFolder("/tmp")
.flatMap(input -> Arrays.asList(input.split( " ")), name("ExtractWords"))
.map(input -> new TimestampedTuple<>(System.currentTimeMillis(), input), name("AddTimestampFn"))
.window(new TimeWindows(Duration.standardMinutes(WINDOW_SIZE)),
new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(1))
.countByKey(input -> new TimestampedTuple<>(input.getTimestamp(), new KeyValPair<>(input.getValue(),
1L ))), name("countWords"))
.map(new FormatAsTableRowFn(), name("FormatAsTableRowFn"))
.print(name("console"))
.populateDag(dag);
• Useful for low latency and high throughput
• Replicates (Partitions) the logic
• Configured at launch time (Application.java or
properties.xml)
• StreamCodec
• Used for controlling distribution of tuples to
downstream partitions
• Unifier (combine results of partitions)
• Passthrough unifier added by platform to merge
results from upstream partitions
• Can also be customized
• Type of partitions
• Static partitions - Statically partition at launch
time
• Dynamic partitions - Partitions changing at
runtime based on latency and/or throughput
• Parallel partitions - Upstream and downstream
operators using same partition scheme
Scalability - Partitioning
23
Scalability - Partitioning (contd.)
24
0 1 2 3
Logical DAG
0 1 2 U
Physical DAG
1
1 2
2
3
Parallel
Partitions
M x N
Partitions
OR
Shuffle
<configuration>
<property>
<name>dt.operator.1. attr.PARTITIONER</name>
<value>com.datatorrent.common.partitioner. StatelessPartitioner:3</value>
</property>
<property>
<name>dt.operator.2.port.inputPortName. attr.PARTITION_PARALLEL</name>
<value>true</value>
</property>
</configuration>
End-to-End Exactly-Once
25
Input Counter Store
Aggregate
CountsWords
Kafka Database
● Input
○ Uses com.datatorrent.contrib.kafka.KafkaSinglePortStringInputOperator
○ Emits words as a stream
○ Operator is idempotent
● Counter
○ com.datatorrent.lib.algo.UniqueCounter
● Store
○ Uses CountStoreOperator
○ Inserts into JDBC
○ Exactly-once results (End-To-End Exactly-once = At-least-once + Idempotency + Consistent State)
https://github.com/DataTorrent/examples/blob/master/tutorials/exactly-once
https://www.datatorrent.com/blog/end-to-end-exactly-once-with-apache-apex/
End-to-End Exactly-Once (Contd.)
26
Input Counter Store
Aggregate
CountsWords
Kafka Database
public static class CountStoreOperator extends AbstractJdbcTransactionableOutputOperator<KeyValPair<String, Integer>>
{
public static final String SQL =
"MERGE INTO words USING (VALUES ?, ?) I (word, wcount)"
+ " ON (words.word=I.word)"
+ " WHEN MATCHED THEN UPDATE SET words.wcount = words.wcount + I.wcount"
+ " WHEN NOT MATCHED THEN INSERT (word, wcount) VALUES (I.word, I.wcount)";
@Override
protected String getUpdateCommand()
{
return SQL;
}
@Override
protected void setStatementParameters(PreparedStatement statement, KeyValPair<String, Integer> tuple)throws SQLException
{
statement.setString(1, tuple.getKey());
statement.setInt(2, tuple.getValue());
}
}
End-to-End Exactly-Once (Contd.)
27
https://www.datatorrent.com/blog/fault-tolerant-file-processing/
Who is using Apex?
28
• Powered by Apex
ᵒ http://apex.apache.org/powered-by-apex.html
ᵒ Also using Apex? Let us know to be added: users@apex.apache.org or @ApacheApex
• Pubmatic
ᵒ https://www.youtube.com/watch?v=JSXpgfQFcU8
• GE
ᵒ https://www.youtube.com/watch?v=hmaSkXhHNu0
ᵒ http://www.slideshare.net/ApacheApex/ge-iot-predix-time-series-data-ingestion-service-usin
g-apache-apex-hadoop
• SilverSpring Networks
ᵒ https://www.youtube.com/watch?v=8VORISKeSjI
ᵒ http://www.slideshare.net/ApacheApex/iot-big-data-ingestion-and-processing-in-hadoop-by-s
ilver-spring-networks
Resources
29
• http://apex.apache.org/
• Learn more - http://apex.apache.org/docs.html
• Subscribe - http://apex.apache.org/community.html
• Download - http://apex.apache.org/downloads.html
• Follow @ApacheApex - https://twitter.com/apacheapex
• Meetups - https://www.meetup.com/topics/apache-apex/
• Examples - https://github.com/DataTorrent/examples
• Slideshare - http://www.slideshare.net/ApacheApex/presentations
• https://www.youtube.com/results?search_query=apache+apex
• Free Enterprise License for Startups -
https://www.datatorrent.com/product/startup-accelerator/
Q&A
30
Ad

Recommended

PPTX
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
Apache Apex
 
PPTX
Kafka to Hadoop Ingest with Parsing, Dedup and other Big Data Transformations
Apache Apex
 
PDF
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Apex
 
PDF
From Batch to Streaming with Apache Apex Dataworks Summit 2017
Apache Apex
 
PPTX
Intro to Apache Apex @ Women in Big Data
Apache Apex
 
PPTX
Java High Level Stream API
Apache Apex
 
PPTX
Ingesting Data from Kafka to JDBC with Transformation and Enrichment
Apache Apex
 
PPTX
Intro to Apache Apex - Next Gen Native Hadoop Platform - Hackac
Apache Apex
 
PPTX
Deep Dive into Apache Apex App Development
Apache Apex
 
PDF
The Future of Apache Storm
DataWorks Summit/Hadoop Summit
 
PPTX
Intro to YARN (Hadoop 2.0) & Apex as YARN App (Next Gen Big Data)
Apache Apex
 
PDF
Actionable Insights with Apache Apex at Apache Big Data 2017 by Devendra Tagare
Apache Apex
 
PDF
Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016
Gyula Fóra
 
PDF
Developing streaming applications with apache apex (strata + hadoop world)
Apache Apex
 
PDF
Large-Scale Stream Processing in the Hadoop Ecosystem
Gyula Fóra
 
PPTX
Hadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache Apex
Apache Apex
 
PPTX
Architectual Comparison of Apache Apex and Spark Streaming
Apache Apex
 
PPTX
Next Gen Big Data Analytics with Apache Apex
DataWorks Summit/Hadoop Summit
 
PPTX
DataTorrent Presentation @ Big Data Application Meetup
Thomas Weise
 
PPTX
Apache Big Data 2016: Next Gen Big Data Analytics with Apache Apex
Apache Apex
 
PPTX
Big Data Berlin v8.0 Stream Processing with Apache Apex
Apache Apex
 
PPTX
Apache Apex: Stream Processing Architecture and Applications
Thomas Weise
 
PPTX
Fault-Tolerant File Input & Output
Apache Apex
 
PDF
Large-Scale Stream Processing in the Hadoop Ecosystem
DataWorks Summit/Hadoop Summit
 
PDF
Low Latency Polyglot Model Scoring using Apache Apex
Apache Apex
 
PPTX
Apache phoenix
Osama Hussein
 
PDF
A TPC Benchmark of Hive LLAP and Comparison with Presto
Yu Liu
 
PPTX
Introduction to Real-Time Data Processing
Apache Apex
 
PPTX
Introduction to Apache Apex
Apache Apex
 
PPTX
HDFS Internals
Apache Apex
 

More Related Content

What's hot (20)

PPTX
Deep Dive into Apache Apex App Development
Apache Apex
 
PDF
The Future of Apache Storm
DataWorks Summit/Hadoop Summit
 
PPTX
Intro to YARN (Hadoop 2.0) & Apex as YARN App (Next Gen Big Data)
Apache Apex
 
PDF
Actionable Insights with Apache Apex at Apache Big Data 2017 by Devendra Tagare
Apache Apex
 
PDF
Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016
Gyula Fóra
 
PDF
Developing streaming applications with apache apex (strata + hadoop world)
Apache Apex
 
PDF
Large-Scale Stream Processing in the Hadoop Ecosystem
Gyula Fóra
 
PPTX
Hadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache Apex
Apache Apex
 
PPTX
Architectual Comparison of Apache Apex and Spark Streaming
Apache Apex
 
PPTX
Next Gen Big Data Analytics with Apache Apex
DataWorks Summit/Hadoop Summit
 
PPTX
DataTorrent Presentation @ Big Data Application Meetup
Thomas Weise
 
PPTX
Apache Big Data 2016: Next Gen Big Data Analytics with Apache Apex
Apache Apex
 
PPTX
Big Data Berlin v8.0 Stream Processing with Apache Apex
Apache Apex
 
PPTX
Apache Apex: Stream Processing Architecture and Applications
Thomas Weise
 
PPTX
Fault-Tolerant File Input & Output
Apache Apex
 
PDF
Large-Scale Stream Processing in the Hadoop Ecosystem
DataWorks Summit/Hadoop Summit
 
PDF
Low Latency Polyglot Model Scoring using Apache Apex
Apache Apex
 
PPTX
Apache phoenix
Osama Hussein
 
PDF
A TPC Benchmark of Hive LLAP and Comparison with Presto
Yu Liu
 
PPTX
Introduction to Real-Time Data Processing
Apache Apex
 
Deep Dive into Apache Apex App Development
Apache Apex
 
The Future of Apache Storm
DataWorks Summit/Hadoop Summit
 
Intro to YARN (Hadoop 2.0) & Apex as YARN App (Next Gen Big Data)
Apache Apex
 
Actionable Insights with Apache Apex at Apache Big Data 2017 by Devendra Tagare
Apache Apex
 
Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016
Gyula Fóra
 
Developing streaming applications with apache apex (strata + hadoop world)
Apache Apex
 
Large-Scale Stream Processing in the Hadoop Ecosystem
Gyula Fóra
 
Hadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache Apex
Apache Apex
 
Architectual Comparison of Apache Apex and Spark Streaming
Apache Apex
 
Next Gen Big Data Analytics with Apache Apex
DataWorks Summit/Hadoop Summit
 
DataTorrent Presentation @ Big Data Application Meetup
Thomas Weise
 
Apache Big Data 2016: Next Gen Big Data Analytics with Apache Apex
Apache Apex
 
Big Data Berlin v8.0 Stream Processing with Apache Apex
Apache Apex
 
Apache Apex: Stream Processing Architecture and Applications
Thomas Weise
 
Fault-Tolerant File Input & Output
Apache Apex
 
Large-Scale Stream Processing in the Hadoop Ecosystem
DataWorks Summit/Hadoop Summit
 
Low Latency Polyglot Model Scoring using Apache Apex
Apache Apex
 
Apache phoenix
Osama Hussein
 
A TPC Benchmark of Hive LLAP and Comparison with Presto
Yu Liu
 
Introduction to Real-Time Data Processing
Apache Apex
 

Viewers also liked (18)

PPTX
Introduction to Apache Apex
Apache Apex
 
PPTX
HDFS Internals
Apache Apex
 
PPTX
Hadoop Interacting with HDFS
Apache Apex
 
PPTX
Capital One's Next Generation Decision in less than 2 ms
Apache Apex
 
PPTX
Introduction to Yarn
Apache Apex
 
PPTX
Intro to Apache Apex (next gen Hadoop) & comparison to Spark Streaming
Apache Apex
 
PPTX
Building Your First Apache Apex (Next Gen Big Data/Hadoop) Application
Apache Apex
 
PPTX
Introduction to Apache Apex and writing a big data streaming application
Apache Apex
 
PPTX
Introduction to Map Reduce
Apache Apex
 
PDF
Apache Hadoop YARN - Enabling Next Generation Data Applications
Hortonworks
 
PPT
Римский корсаков снегурочка
Ninel Kek
 
PPT
Цветочные легенды
Ninel Kek
 
PPTX
High Performance Distributed Systems with CQRS
Jonathan Oliver
 
PPTX
правописание приставок урок№4
HomichAlla
 
PPTX
бсп (обоб. урок)
HomichAlla
 
PDF
Troubleshooting mysql-tutorial
james tong
 
PDF
Towards True Elasticity of Spark-(Michael Le and Min Li, IBM)
Spark Summit
 
PDF
Windowing in Apache Apex
Apache Apex
 
Introduction to Apache Apex
Apache Apex
 
HDFS Internals
Apache Apex
 
Hadoop Interacting with HDFS
Apache Apex
 
Capital One's Next Generation Decision in less than 2 ms
Apache Apex
 
Introduction to Yarn
Apache Apex
 
Intro to Apache Apex (next gen Hadoop) & comparison to Spark Streaming
Apache Apex
 
Building Your First Apache Apex (Next Gen Big Data/Hadoop) Application
Apache Apex
 
Introduction to Apache Apex and writing a big data streaming application
Apache Apex
 
Introduction to Map Reduce
Apache Apex
 
Apache Hadoop YARN - Enabling Next Generation Data Applications
Hortonworks
 
Римский корсаков снегурочка
Ninel Kek
 
Цветочные легенды
Ninel Kek
 
High Performance Distributed Systems with CQRS
Jonathan Oliver
 
правописание приставок урок№4
HomichAlla
 
бсп (обоб. урок)
HomichAlla
 
Troubleshooting mysql-tutorial
james tong
 
Towards True Elasticity of Spark-(Michael Le and Min Li, IBM)
Spark Summit
 
Windowing in Apache Apex
Apache Apex
 
Ad

Similar to Apache Big Data EU 2016: Building Streaming Applications with Apache Apex (20)

PDF
Stream Processing use cases and applications with Apache Apex by Thomas Weise
Big Data Spain
 
PDF
BigDataSpain 2016: Stream Processing Applications with Apache Apex
Thomas Weise
 
PDF
Introduction to Apache Apex by Thomas Weise
Big Data Spain
 
PDF
Building Your First Apache Apex Application
Apache Apex
 
PDF
Building your first aplication using Apache Apex
Yogi Devendra Vyavahare
 
PPTX
Introduction to Apache Flink
mxmxm
 
PPTX
Thomas Weise, Apache Apex PMC Member and Architect/Co-Founder, DataTorrent - ...
Dataconomy Media
 
PPTX
Apache Apex: Stream Processing Architecture and Applications
Comsysto Reply GmbH
 
PDF
BigDataSpain 2016: Introduction to Apache Apex
Thomas Weise
 
PPTX
Stream processing - Apache flink
Renato Guimaraes
 
PPTX
Ingestion and Dimensions Compute and Enrich using Apache Apex
Apache Apex
 
PDF
It's Time To Stop Using Lambda Architecture
Yaroslav Tkachenko
 
PDF
Streaming architecture patterns
hadooparchbook
 
PDF
Building Scalable Data Pipelines - 2016 DataPalooza Seattle
Evan Chan
 
PPTX
Apache Flink Training: System Overview
Flink Forward
 
PDF
Real Time Big Data Management
Albert Bifet
 
PPTX
Flink Streaming
Gyula Fóra
 
PDF
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
confluent
 
PPTX
Stream Processing with Apache Apex
Pramod Immaneni
 
PPTX
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
Fabian Hueske
 
Stream Processing use cases and applications with Apache Apex by Thomas Weise
Big Data Spain
 
BigDataSpain 2016: Stream Processing Applications with Apache Apex
Thomas Weise
 
Introduction to Apache Apex by Thomas Weise
Big Data Spain
 
Building Your First Apache Apex Application
Apache Apex
 
Building your first aplication using Apache Apex
Yogi Devendra Vyavahare
 
Introduction to Apache Flink
mxmxm
 
Thomas Weise, Apache Apex PMC Member and Architect/Co-Founder, DataTorrent - ...
Dataconomy Media
 
Apache Apex: Stream Processing Architecture and Applications
Comsysto Reply GmbH
 
BigDataSpain 2016: Introduction to Apache Apex
Thomas Weise
 
Stream processing - Apache flink
Renato Guimaraes
 
Ingestion and Dimensions Compute and Enrich using Apache Apex
Apache Apex
 
It's Time To Stop Using Lambda Architecture
Yaroslav Tkachenko
 
Streaming architecture patterns
hadooparchbook
 
Building Scalable Data Pipelines - 2016 DataPalooza Seattle
Evan Chan
 
Apache Flink Training: System Overview
Flink Forward
 
Real Time Big Data Management
Albert Bifet
 
Flink Streaming
Gyula Fóra
 
Kafka Summit NYC 2017 - Building Advanced Streaming Applications using the La...
confluent
 
Stream Processing with Apache Apex
Pramod Immaneni
 
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
Fabian Hueske
 
Ad

Recently uploaded (20)

PDF
Tech-ASan: Two-stage check for Address Sanitizer - Yixuan Cao.pdf
caoyixuan2019
 
PPTX
OpenACC and Open Hackathons Monthly Highlights June 2025
OpenACC
 
PPTX
Security Tips for Enterprise Azure Solutions
Michele Leroux Bustamante
 
PPTX
You are not excused! How to avoid security blind spots on the way to production
Michele Leroux Bustamante
 
PDF
Lessons Learned from Developing Secure AI Workflows.pdf
Priyanka Aash
 
PDF
Hyderabad MuleSoft In-Person Meetup (June 21, 2025) Slides
Ravi Tamada
 
PPTX
" How to survive with 1 billion vectors and not sell a kidney: our low-cost c...
Fwdays
 
PDF
Cracking the Code - Unveiling Synergies Between Open Source Security and AI.pdf
Priyanka Aash
 
PDF
cnc-processing-centers-centateq-p-110-en.pdf
AmirStern2
 
DOCX
Daily Lesson Log MATATAG ICT TEchnology 8
LOIDAALMAZAN3
 
PDF
Mastering AI Workflows with FME by Mark Döring
Safe Software
 
PDF
Database Benchmarking for Performance Masterclass: Session 2 - Data Modeling ...
ScyllaDB
 
PPTX
UserCon Belgium: Honey, VMware increased my bill
stijn40
 
PDF
Connecting Data and Intelligence: The Role of FME in Machine Learning
Safe Software
 
PDF
Enhance GitHub Copilot using MCP - Enterprise version.pdf
Nilesh Gule
 
PDF
The Growing Value and Application of FME & GenAI
Safe Software
 
PDF
Coordinated Disclosure for ML - What's Different and What's the Same.pdf
Priyanka Aash
 
PDF
ReSTIR [DI]: Spatiotemporal reservoir resampling for real-time ray tracing ...
revolcs10
 
PDF
Quantum AI: Where Impossible Becomes Probable
Saikat Basu
 
PDF
Smarter Aviation Data Management: Lessons from Swedavia Airports and Sweco
Safe Software
 
Tech-ASan: Two-stage check for Address Sanitizer - Yixuan Cao.pdf
caoyixuan2019
 
OpenACC and Open Hackathons Monthly Highlights June 2025
OpenACC
 
Security Tips for Enterprise Azure Solutions
Michele Leroux Bustamante
 
You are not excused! How to avoid security blind spots on the way to production
Michele Leroux Bustamante
 
Lessons Learned from Developing Secure AI Workflows.pdf
Priyanka Aash
 
Hyderabad MuleSoft In-Person Meetup (June 21, 2025) Slides
Ravi Tamada
 
" How to survive with 1 billion vectors and not sell a kidney: our low-cost c...
Fwdays
 
Cracking the Code - Unveiling Synergies Between Open Source Security and AI.pdf
Priyanka Aash
 
cnc-processing-centers-centateq-p-110-en.pdf
AmirStern2
 
Daily Lesson Log MATATAG ICT TEchnology 8
LOIDAALMAZAN3
 
Mastering AI Workflows with FME by Mark Döring
Safe Software
 
Database Benchmarking for Performance Masterclass: Session 2 - Data Modeling ...
ScyllaDB
 
UserCon Belgium: Honey, VMware increased my bill
stijn40
 
Connecting Data and Intelligence: The Role of FME in Machine Learning
Safe Software
 
Enhance GitHub Copilot using MCP - Enterprise version.pdf
Nilesh Gule
 
The Growing Value and Application of FME & GenAI
Safe Software
 
Coordinated Disclosure for ML - What's Different and What's the Same.pdf
Priyanka Aash
 
ReSTIR [DI]: Spatiotemporal reservoir resampling for real-time ray tracing ...
revolcs10
 
Quantum AI: Where Impossible Becomes Probable
Saikat Basu
 
Smarter Aviation Data Management: Lessons from Swedavia Airports and Sweco
Safe Software
 

Apache Big Data EU 2016: Building Streaming Applications with Apache Apex

  • 1. Building Streaming Applications with Apache Apex Chinmay Kolhatkar, Committer @ApacheApex, Engineer @DataTorrent Thomas Weise, PMC Chair @ApacheApex, Architect @DataTorrent Nov 15th 2016
  • 2. Agenda 2 • Application Development Model • Creating Apex Application - Project Structure • Apex APIs • Configuration Example • Operator APIs • Overview of Operator Library • Frequently used Connectors • Stateful Transformation & Windowing • Scalability - Partitioning • End-to-end Exactly Once
  • 3. Application Development Model 3 ▪Stream is a sequence of data tuples ▪Operator takes one or more input streams, performs computations & emits one or more output streams • Each Operator is YOUR custom business logic in java, or built-in operator from our open source library • Operator has many instances that run in parallel and each instance is single-threaded ▪Directed Acyclic Graph (DAG) is made up of operators and streams Directed Acyclic Graph (DAG) Filtered Stream Output Stream Tuple Tuple FilteredStream Enriched Stream Enriched Stream er Operator er Operator er Operator er Operator er Operator er Operator
  • 4. Creating Apex Application Project 4 chinmay@chinmay-VirtualBox:~/src$ mvn archetype:generate -DarchetypeGroupId=org.apache.apex -DarchetypeArtifactId=apex-app-archetype -DarchetypeVersion=LATEST -DgroupId=com.example -Dpackage=com.example.myapexapp -DartifactId=myapexapp -Dversion=1.0-SNAPSHOT … … ... Confirm properties configuration: groupId: com.example artifactId: myapexapp version: 1.0-SNAPSHOT package: com.example.myapexapp archetypeVersion: LATEST Y: : Y … … ... [INFO] project created from Archetype in dir: /media/sf_workspace/src/myapexapp [INFO] ------------------------------------------------------------------------ [INFO] BUILD SUCCESS [INFO] ------------------------------------------------------------------------ [INFO] Total time: 13.141 s [INFO] Finished at: 2016-11-15T14:06:56+05:30 [INFO] Final Memory: 18M/216M [INFO] ------------------------------------------------------------------------ chinmay@chinmay-VirtualBox:~/src$ https://www.youtube.com/watch?v=z-eeh-tjQrc
  • 5. Apex Application Project Structure 5 • pom.xml • Defines project structure and dependencies • Application.java • Defines the DAG • RandomNumberGenerator.java • Sample Operator • properties.xml • Contains operator and application properties and attributes • ApplicationTest.java • Sample test to test application in local mode
  • 6. Apex APIs: Compositional (Low level) 6 Input Parser Counter Output CountsWordsLines Kafka Database Filter Filtered
  • 7. Apex APIs: Declarative (High Level) 7 File Input Parser Word Counter Console Output CountsWordsLines Folder StdOut StreamFactory.fromFolder("/tmp") .flatMap(input -> Arrays.asList(input.split(" ")), name("Words")) .window(new WindowOption.GlobalWindow(), new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(1)) .countByKey(input -> new Tuple.PlainTuple<>(new KeyValPair<>(input, 1L)), name("countByKey")) .map(input -> input.getValue(), name("Counts")) .print(name("Console")) .populateDag(dag);
  • 8. Apex APIs: SQL 8 Kafka Input CSV Parser Filter CSV Formattter FilteredWordsLines Kafka File Project Projected Line Writer Formatted SQLExecEnvironment.getEnvironment() .registerTable("ORDERS", new KafkaEndpoint(conf.get("broker"), conf.get("topic"), new CSVMessageFormat(conf.get("schemaInDef")))) .registerTable("SALES", new FileEndpoint(conf.get("destFolder"), conf.get("destFileName"), new CSVMessageFormat(conf.get("schemaOutDef")))) .registerFunction("APEXCONCAT", this.getClass(), "apex_concat_str") .executeSQL(dag, "INSERT INTO SALES " + "SELECT STREAM ROWTIME, FLOOR(ROWTIME TO DAY), APEXCONCAT('OILPAINT', SUBSTRING(PRODUCT, 6, 7) " + "FROM ORDERS WHERE ID > 3 AND PRODUCT LIKE 'paint%'");
  • 9. Apex APIs: Beam 9 • Apex Runner of Beam is available!! • Build once run-anywhere model • Beam Streaming applications can be run on apex runner: public static void main(String[] args) { Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); // Run with Apex runner options.setRunner(ApexRunner.class); Pipeline p = Pipeline.create(options); p.apply("ReadLines", TextIO.Read.from(options.getInput())) .apply(new CountWords()) .apply(MapElements.via(new FormatAsTextFn())) .apply("WriteCounts", TextIO.Write.to(options.getOutput())); .run().waitUntilFinish(); }
  • 10. Apex APIs: SAMOA 10 • Build once run-anywhere model for online machine learning algorithms • Any machine learning algorithm present in SAMOA can be run directly on Apex. • Uses Apex Iteration Support • Following example does classification of input data from HDFS using VHT algorithm on Apex: $ bin/samoa apex ../SAMOA-Apex-0.4.0-incubating-SNAPSHOT.jar "PrequentialEvaluation -d /tmp/dump.csv -l (classifiers.trees.VerticalHoeffdingTree -p 1) -s (org.apache.samoa.streams.ArffFileStream -s HDFSFileStreamSource -f /tmp/user/input/covtypeNorm.arff)"
  • 11. Configuration (properties.xml) 11 Input Parser Counter Output CountsWordsLines Kafka Database Filter Filtered
  • 12. Streaming Window Processing Time Window 12 • Finite time sliced windows based on processing (event arrival) time • Used for bookkeeping of streaming application • Derived Windows are: Checkpoint Windows, Committed Windows
  • 13. Operator APIs 13 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()
  • 14. Overview of Operator Library (Malhar) 14 RDBMS • JDBC • MySQL • Oracle • MemSQL NoSQL • Cassandra, HBase • Aerospike, Accumulo • Couchbase/ CouchDB • Redis, MongoDB • Geode Messaging • Kafka • JMS (ActiveMQ etc.) • Kinesis, SQS • Flume, NiFi 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 • MQTT • SMTP Other • Elastic Search • Script (JavaScript, Python, R) • Solr • Twitter
  • 15. Frequently used Connectors Kafka Input 15 KafkaSinglePortInputOperator KafkaSinglePortByteArrayInputOperator Library malhar-contrib malhar-kafka Kafka Consumer 0.8 0.9 Emit Type byte[] byte[] Fault-Tolerance At Least Once, Exactly Once At Least Once, Exactly Once Scalability Static and Dynamic (with Kafka metadata) Static and Dynamic (with Kafka metadata) Multi-Cluster/Topic Yes Yes Idempotent Yes Yes Partition Strategy 1:1, 1:M 1:1, 1:M
  • 16. Frequently used Connectors Kafka Output 16 KafkaSinglePortOutputOperator KafkaSinglePortExactlyOnceOutputOperator Library malhar-contrib malhar-kafka Kafka Producer 0.8 0.9 Fault-Tolerance At Least Once At Least Once, Exactly Once Scalability Static and Dynamic (with Kafka metadata) Static and Dynamic, Automatic Partitioning based on Kafka metadata Multi-Cluster/Topic Yes Yes Idempotent Yes Yes Partition Strategy 1:1, 1:M 1:1, 1:M
  • 17. Frequently used Connectors File Input 17 • AbstractFileInputOperator • Used to read a file from source and emit the content of the file to downstream operator • Operator is idempotent • Supports Partitioning • Few Concrete Impl • FileLineInputOperator • AvroFileInputOperator • ParquetFilePOJOReader • https://www.datatorrent.com/blog/f ault-tolerant-file-processing/
  • 18. Frequently used Connectors File Output 18 • AbstractFileOutputOperator • Writes data to a file • Supports Partitions • Exactly-once results • Upstream operators should be idempotent • Few Concrete Impl • StringFileOutputOperator • https://www.datatorrent.com/blog/f ault-tolerant-file-processing/
  • 19. Windowing Support 19 • Event-time Windows • Computation based on event-time present in the tuple • Types of event-time windows supported: • Global : Single event-time window throughout the lifecycle of application • Timed : Tuple is assigned to single, non-overlapping, fixed width windows immediately followed by next window • Sliding Time : Tuple is can be assigned to multiple, overlapping fixed width windows. • Session : Tuple is assigned to single, variable width windows with a predefined min gap
  • 20. Stateful Windowed Processing 20 • WindowedOperator from malhar-library • Used to process data based on Event time as contrary to ingression time • Supports windowing semantics of Apache Beam model • Supported features: • Watermarks • Allowed Lateness • Accumulation • Accumulation Modes: Accumulating, Discarding, Accumulating & Retracting • Triggers • Storage • In memory based • Managed State based
  • 21. Stateful Windowed Processing Compositional API 21 @Override public void populateDAG(DAG dag, Configuration configuration) { WordGenerator inputOperator = new WordGenerator(); KeyedWindowedOperatorImpl windowedOperator = new KeyedWindowedOperatorImpl(); Accumulation<Long, MutableLong, Long> sum = new SumAccumulation(); windowedOperator.setAccumulation(sum); windowedOperator.setDataStorage(new InMemoryWindowedKeyedStorage<String, MutableLong>()); windowedOperator.setRetractionStorage(new InMemoryWindowedKeyedStorage<String, Long>()); windowedOperator.setWindowStateStorage(new InMemoryWindowedStorage<WindowState>()); windowedOperator.setWindowOption(new WindowOption.TimeWindows(Duration.standardMinutes(1))); windowedOperator.setTriggerOption(TriggerOption.AtWatermark() .withEarlyFiringsAtEvery(Duration.millis(1000)) .accumulatingAndRetractingFiredPanes()); windowedOperator.setAllowedLateness(Duration.millis(14000)); ConsoleOutputOperator outputOperator = new ConsoleOutputOperator(); dag.addOperator( "inputOperator", inputOperator); dag.addOperator( "windowedOperator", windowedOperator); dag.addOperator( "outputOperator", outputOperator); dag.addStream( "input_windowed", inputOperator. output, windowedOperator.input); dag.addStream( "windowed_output", windowedOperator.output, outputOperator. input); }
  • 22. Stateful Windowed Processing Declarative API 22 StreamFactory.fromFolder("/tmp") .flatMap(input -> Arrays.asList(input.split( " ")), name("ExtractWords")) .map(input -> new TimestampedTuple<>(System.currentTimeMillis(), input), name("AddTimestampFn")) .window(new TimeWindows(Duration.standardMinutes(WINDOW_SIZE)), new TriggerOption().accumulatingFiredPanes().withEarlyFiringsAtEvery(1)) .countByKey(input -> new TimestampedTuple<>(input.getTimestamp(), new KeyValPair<>(input.getValue(), 1L ))), name("countWords")) .map(new FormatAsTableRowFn(), name("FormatAsTableRowFn")) .print(name("console")) .populateDag(dag);
  • 23. • Useful for low latency and high throughput • Replicates (Partitions) the logic • Configured at launch time (Application.java or properties.xml) • StreamCodec • Used for controlling distribution of tuples to downstream partitions • Unifier (combine results of partitions) • Passthrough unifier added by platform to merge results from upstream partitions • Can also be customized • Type of partitions • Static partitions - Statically partition at launch time • Dynamic partitions - Partitions changing at runtime based on latency and/or throughput • Parallel partitions - Upstream and downstream operators using same partition scheme Scalability - Partitioning 23
  • 24. Scalability - Partitioning (contd.) 24 0 1 2 3 Logical DAG 0 1 2 U Physical DAG 1 1 2 2 3 Parallel Partitions M x N Partitions OR Shuffle <configuration> <property> <name>dt.operator.1. attr.PARTITIONER</name> <value>com.datatorrent.common.partitioner. StatelessPartitioner:3</value> </property> <property> <name>dt.operator.2.port.inputPortName. attr.PARTITION_PARALLEL</name> <value>true</value> </property> </configuration>
  • 25. End-to-End Exactly-Once 25 Input Counter Store Aggregate CountsWords Kafka Database ● Input ○ Uses com.datatorrent.contrib.kafka.KafkaSinglePortStringInputOperator ○ Emits words as a stream ○ Operator is idempotent ● Counter ○ com.datatorrent.lib.algo.UniqueCounter ● Store ○ Uses CountStoreOperator ○ Inserts into JDBC ○ Exactly-once results (End-To-End Exactly-once = At-least-once + Idempotency + Consistent State) https://github.com/DataTorrent/examples/blob/master/tutorials/exactly-once https://www.datatorrent.com/blog/end-to-end-exactly-once-with-apache-apex/
  • 26. End-to-End Exactly-Once (Contd.) 26 Input Counter Store Aggregate CountsWords Kafka Database public static class CountStoreOperator extends AbstractJdbcTransactionableOutputOperator<KeyValPair<String, Integer>> { public static final String SQL = "MERGE INTO words USING (VALUES ?, ?) I (word, wcount)" + " ON (words.word=I.word)" + " WHEN MATCHED THEN UPDATE SET words.wcount = words.wcount + I.wcount" + " WHEN NOT MATCHED THEN INSERT (word, wcount) VALUES (I.word, I.wcount)"; @Override protected String getUpdateCommand() { return SQL; } @Override protected void setStatementParameters(PreparedStatement statement, KeyValPair<String, Integer> tuple)throws SQLException { statement.setString(1, tuple.getKey()); statement.setInt(2, tuple.getValue()); } }
  • 28. Who is using Apex? 28 • Powered by Apex ᵒ http://apex.apache.org/powered-by-apex.html ᵒ Also using Apex? Let us know to be added: [email protected] or @ApacheApex • Pubmatic ᵒ https://www.youtube.com/watch?v=JSXpgfQFcU8 • GE ᵒ https://www.youtube.com/watch?v=hmaSkXhHNu0 ᵒ http://www.slideshare.net/ApacheApex/ge-iot-predix-time-series-data-ingestion-service-usin g-apache-apex-hadoop • SilverSpring Networks ᵒ https://www.youtube.com/watch?v=8VORISKeSjI ᵒ http://www.slideshare.net/ApacheApex/iot-big-data-ingestion-and-processing-in-hadoop-by-s ilver-spring-networks
  • 29. Resources 29 • http://apex.apache.org/ • Learn more - http://apex.apache.org/docs.html • Subscribe - http://apex.apache.org/community.html • Download - http://apex.apache.org/downloads.html • Follow @ApacheApex - https://twitter.com/apacheapex • Meetups - https://www.meetup.com/topics/apache-apex/ • Examples - https://github.com/DataTorrent/examples • Slideshare - http://www.slideshare.net/ApacheApex/presentations • https://www.youtube.com/results?search_query=apache+apex • Free Enterprise License for Startups - https://www.datatorrent.com/product/startup-accelerator/