SlideShare une entreprise Scribd logo
1  sur  30
Télécharger pour lire hors ligne
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

Contenu connexe

Tendances

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

Tendances (20)

Deep Dive into Apache Apex App Development
Deep Dive into Apache Apex App DevelopmentDeep Dive into Apache Apex App Development
Deep Dive into Apache Apex App Development
 
The Future of Apache Storm
The Future of Apache StormThe Future of Apache Storm
The Future of Apache Storm
 
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)
 
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
 
Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016
Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016
Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016
 
Developing streaming applications with apache apex (strata + hadoop world)
Developing streaming applications with apache apex (strata + hadoop world)Developing streaming applications with apache apex (strata + hadoop world)
Developing streaming applications with apache apex (strata + hadoop world)
 
Large-Scale Stream Processing in the Hadoop Ecosystem
Large-Scale Stream Processing in the Hadoop EcosystemLarge-Scale Stream Processing in the Hadoop Ecosystem
Large-Scale Stream Processing in the Hadoop Ecosystem
 
Hadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache Apex
Hadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache ApexHadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache Apex
Hadoop Summit SJ 2016: Next Gen Big Data Analytics with Apache Apex
 
Architectual Comparison of Apache Apex and Spark Streaming
Architectual Comparison of Apache Apex and Spark StreamingArchitectual Comparison of Apache Apex and Spark Streaming
Architectual Comparison of Apache Apex and Spark Streaming
 
Next Gen Big Data Analytics with Apache Apex
Next Gen Big Data Analytics with Apache Apex Next Gen Big Data Analytics with Apache Apex
Next Gen Big Data Analytics with Apache Apex
 
DataTorrent Presentation @ Big Data Application Meetup
DataTorrent Presentation @ Big Data Application MeetupDataTorrent Presentation @ Big Data Application Meetup
DataTorrent Presentation @ Big Data Application Meetup
 
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
 
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: 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
 
Fault-Tolerant File Input & Output
Fault-Tolerant File Input & OutputFault-Tolerant File Input & Output
Fault-Tolerant File Input & Output
 
Large-Scale Stream Processing in the Hadoop Ecosystem
Large-Scale Stream Processing in the Hadoop Ecosystem Large-Scale Stream Processing in the Hadoop Ecosystem
Large-Scale Stream Processing in the Hadoop Ecosystem
 
Low Latency Polyglot Model Scoring using Apache Apex
Low Latency Polyglot Model Scoring using Apache ApexLow Latency Polyglot Model Scoring using Apache Apex
Low Latency Polyglot Model Scoring using Apache Apex
 
Apache phoenix
Apache phoenixApache phoenix
Apache phoenix
 
A TPC Benchmark of Hive LLAP and Comparison with Presto
A TPC Benchmark of Hive LLAP and Comparison with PrestoA TPC Benchmark of Hive LLAP and Comparison with Presto
A TPC Benchmark of Hive LLAP and Comparison with Presto
 
Introduction to Real-Time Data Processing
Introduction to Real-Time Data ProcessingIntroduction to Real-Time Data Processing
Introduction to Real-Time Data Processing
 

En vedette

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
 
Римский корсаков снегурочка
Римский корсаков снегурочкаРимский корсаков снегурочка
Римский корсаков снегурочка
Ninel Kek
 
Цветочные легенды
Цветочные легендыЦветочные легенды
Цветочные легенды
Ninel Kek
 
High Performance Distributed Systems with CQRS
High Performance Distributed Systems with CQRSHigh Performance Distributed Systems with CQRS
High Performance Distributed Systems with CQRS
Jonathan Oliver
 
правописание приставок урок№4
правописание приставок урок№4правописание приставок урок№4
правописание приставок урок№4
HomichAlla
 

En vedette (18)

Introduction to Apache Apex
Introduction to Apache ApexIntroduction to Apache Apex
Introduction to Apache Apex
 
HDFS Internals
HDFS InternalsHDFS Internals
HDFS Internals
 
Hadoop Interacting with HDFS
Hadoop Interacting with HDFSHadoop Interacting with HDFS
Hadoop Interacting with HDFS
 
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
 
Introduction to Yarn
Introduction to YarnIntroduction to Yarn
Introduction to Yarn
 
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
 
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
 
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 Map Reduce
Introduction to Map ReduceIntroduction to Map Reduce
Introduction to Map Reduce
 
Apache Hadoop YARN - Enabling Next Generation Data Applications
Apache Hadoop YARN - Enabling Next Generation Data ApplicationsApache Hadoop YARN - Enabling Next Generation Data Applications
Apache Hadoop YARN - Enabling Next Generation Data Applications
 
Римский корсаков снегурочка
Римский корсаков снегурочкаРимский корсаков снегурочка
Римский корсаков снегурочка
 
Цветочные легенды
Цветочные легендыЦветочные легенды
Цветочные легенды
 
High Performance Distributed Systems with CQRS
High Performance Distributed Systems with CQRSHigh Performance Distributed Systems with CQRS
High Performance Distributed Systems with CQRS
 
правописание приставок урок№4
правописание приставок урок№4правописание приставок урок№4
правописание приставок урок№4
 
бсп (обоб. урок)
бсп (обоб. урок)бсп (обоб. урок)
бсп (обоб. урок)
 
Troubleshooting mysql-tutorial
Troubleshooting mysql-tutorialTroubleshooting mysql-tutorial
Troubleshooting mysql-tutorial
 
Towards True Elasticity of Spark-(Michael Le and Min Li, IBM)
Towards True Elasticity of Spark-(Michael Le and Min Li, IBM)Towards True Elasticity of Spark-(Michael Le and Min Li, IBM)
Towards True Elasticity of Spark-(Michael Le and Min Li, IBM)
 
Windowing in Apache Apex
Windowing in Apache ApexWindowing in Apache Apex
Windowing in Apache Apex
 

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

Flink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseFlink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San Jose
Kostas Tzoumas
 
Architectures, Frameworks and Infrastructure
Architectures, Frameworks and InfrastructureArchitectures, Frameworks and Infrastructure
Architectures, Frameworks and Infrastructure
harendra_pathak
 

Similaire à Apache Big Data EU 2016: Building Streaming Applications with Apache Apex (20)

BigDataSpain 2016: Stream Processing Applications with Apache Apex
BigDataSpain 2016: Stream Processing Applications with Apache ApexBigDataSpain 2016: Stream Processing Applications with Apache Apex
BigDataSpain 2016: Stream Processing Applications with Apache Apex
 
Stream Processing use cases and applications with Apache Apex by Thomas Weise
Stream Processing use cases and applications with Apache Apex by Thomas WeiseStream Processing use cases and applications with Apache Apex by Thomas Weise
Stream Processing use cases and applications with Apache Apex by Thomas Weise
 
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
 
Ingesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmedIngesting hdfs intosolrusingsparktrimmed
Ingesting hdfs intosolrusingsparktrimmed
 
Using existing language skillsets to create large-scale, cloud-based analytics
Using existing language skillsets to create large-scale, cloud-based analyticsUsing existing language skillsets to create large-scale, cloud-based analytics
Using existing language skillsets to create large-scale, cloud-based analytics
 
Flink history, roadmap and vision
Flink history, roadmap and visionFlink history, roadmap and vision
Flink history, roadmap and vision
 
Running Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on HadoopRunning Airflow Workflows as ETL Processes on Hadoop
Running Airflow Workflows as ETL Processes on Hadoop
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
 
20170126 big data processing
20170126 big data processing20170126 big data processing
20170126 big data processing
 
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
 
Building Scalable Data Pipelines - 2016 DataPalooza Seattle
Building Scalable Data Pipelines - 2016 DataPalooza SeattleBuilding Scalable Data Pipelines - 2016 DataPalooza Seattle
Building Scalable Data Pipelines - 2016 DataPalooza Seattle
 
Past, Present, and Future of Apache Storm
Past, Present, and Future of Apache StormPast, Present, and Future of Apache Storm
Past, Present, and Future of Apache Storm
 
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
Spark (Structured) Streaming vs. Kafka Streams - two stream processing platfo...
 
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data ProcessingApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
 
Flink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseFlink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San Jose
 
Deploying Apache Flume to enable low-latency analytics
Deploying Apache Flume to enable low-latency analyticsDeploying Apache Flume to enable low-latency analytics
Deploying Apache Flume to enable low-latency analytics
 
Apache Flink: Past, Present and Future
Apache Flink: Past, Present and FutureApache Flink: Past, Present and Future
Apache Flink: Past, Present and Future
 
Data streaming-systems
Data streaming-systemsData streaming-systems
Data streaming-systems
 
Hadoop and HBase experiences in perf log project
Hadoop and HBase experiences in perf log projectHadoop and HBase experiences in perf log project
Hadoop and HBase experiences in perf log project
 
Architectures, Frameworks and Infrastructure
Architectures, Frameworks and InfrastructureArchitectures, Frameworks and Infrastructure
Architectures, Frameworks and Infrastructure
 

Plus de Apache Apex

Plus de Apache Apex (6)

Intro to Big Data Hadoop
Intro to Big Data HadoopIntro to Big Data Hadoop
Intro to Big Data Hadoop
 
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 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
 

Dernier

Histor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slideHistor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slide
vu2urc
 
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptxEIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
Earley Information Science
 
IAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI SolutionsIAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI Solutions
Enterprise Knowledge
 

Dernier (20)

Exploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone ProcessorsExploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone Processors
 
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
 
[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdf[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdf
 
08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking Men08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking Men
 
Histor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slideHistor y of HAM Radio presentation slide
Histor y of HAM Radio presentation slide
 
Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...
 
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
 
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptxEIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
EIS-Webinar-Prompt-Knowledge-Eng-2024-04-08.pptx
 
Real Time Object Detection Using Open CV
Real Time Object Detection Using Open CVReal Time Object Detection Using Open CV
Real Time Object Detection Using Open CV
 
GenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day PresentationGenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day Presentation
 
Scaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationScaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organization
 
The Codex of Business Writing Software for Real-World Solutions 2.pptx
The Codex of Business Writing Software for Real-World Solutions 2.pptxThe Codex of Business Writing Software for Real-World Solutions 2.pptx
The Codex of Business Writing Software for Real-World Solutions 2.pptx
 
How to convert PDF to text with Nanonets
How to convert PDF to text with NanonetsHow to convert PDF to text with Nanonets
How to convert PDF to text with Nanonets
 
Advantages of Hiring UIUX Design Service Providers for Your Business
Advantages of Hiring UIUX Design Service Providers for Your BusinessAdvantages of Hiring UIUX Design Service Providers for Your Business
Advantages of Hiring UIUX Design Service Providers for Your Business
 
Understanding Discord NSFW Servers A Guide for Responsible Users.pdf
Understanding Discord NSFW Servers A Guide for Responsible Users.pdfUnderstanding Discord NSFW Servers A Guide for Responsible Users.pdf
Understanding Discord NSFW Servers A Guide for Responsible Users.pdf
 
Finology Group – Insurtech Innovation Award 2024
Finology Group – Insurtech Innovation Award 2024Finology Group – Insurtech Innovation Award 2024
Finology Group – Insurtech Innovation Award 2024
 
Data Cloud, More than a CDP by Matt Robison
Data Cloud, More than a CDP by Matt RobisonData Cloud, More than a CDP by Matt Robison
Data Cloud, More than a CDP by Matt Robison
 
Boost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivityBoost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivity
 
IAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI SolutionsIAC 2024 - IA Fast Track to Search Focused AI Solutions
IAC 2024 - IA Fast Track to Search Focused AI Solutions
 
08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking Men08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking Men
 

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: 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
  • 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/