SlideShare ist ein Scribd-Unternehmen logo
1 von 31
Downloaden Sie, um offline zu lesen
1
Building Streaming Applications with
Apache Apex
Thomas Weise <thw@apache.org> @thweise
PMC Chair Apache Apex, Architect DataTorrent
Big Data Spain, Madrid, Nov 18th 2016
Agenda
3
• Application Development Model
• Creating Apex Application - Project S tructure
• Apex APIs
• Configuration Example
• Operator APIs
• Overview of Operator Library
• Frequently used Connectors
• S tateful Transformation & Windowing
• S calability – Partitioning
• End-to-end Exactly Once
Application Development Model
4
▪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)
Output
Stream
Tupl
e
Tupl
e
er
Operator
er
Operator
er
Operator
er
Operator
er
Operator
er
Operator
Creating Apex Application Project
5
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
6
• pom.xml
•Defines project structure and
dependencies
•Application.java
•Defines the DAG
•RandomNumberGenerator.java
•S ample Operator
•properties.xml
•Contains operator and application
properties and attributes
•ApplicationTest.java
•S ample test to test application in local
mode
APIs: Compositional (Low level)
7
Input Parser Counter Output
CountsWordsLines
Kafka Database
Filter
Filtered
Apex APIs: Declarative (High Level)
8
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);
APIs: S QL
9
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%'");
APIs: Beam
10
• Apex Runner for Apache Beam is now available!!
•Build once run-anywhere model
•Beam S treaming 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();
}
APIs: S AMOA
11
•Build once run-anywhere model for online machine learning algorithms
•Any machine learning algorithm present in S AMOA can be run directly on Apex.
•Uses Apex Iteration S upport
•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)
12
Input Parser Counter Output
CountsWordsLines
Kafka Database
Filter
Filtered
Streaming Window
Processing Time Window
13
•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
14
Next
streaming
window
Next
streaming
window
Input Adapters - S tarting 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 Library
15
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
16
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
17
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
18
•AbstractFileInputOperator
•Used to read a file from source and
emit the content of the file to
downstream operator
•Operator is idempotent
•S upports Partitioning
•Few Concrete Impl
•FileLineInputOperator
•AvroFileInputOperator
•ParquetFilePOJ OReader
•https://www.datatorrent.com/blog/f
ault-tolerant-file-processing/
Frequently used Connectors
File Output
19
•AbstractFileOutputOperator
•Writes data to a file
•S upports Partitions
•Exactly-once results
•Upstream operators should be
idempotent
•Few Concrete Impl
•S tringFileOutputOperator
•https://www.datatorrent.com/blog/f
ault-tolerant-file-processing/
Windowing Support
20
• Event-time Windows
• Computation based on event-time present in the tuple
• Types of event-time windows:
ᵒ Global : S ingle 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 predefined min gap
Stateful Windowed Processing
21
• WindowedOperator (org.apache.apex:malhar-library)
• Used to process data based on Event time as contrary to ingression time
• S upports windowing semantics of Apache Beam model
• Features:
ᵒ Watermarks
ᵒ Allowed Lateness
ᵒ Accumulation
ᵒ Accumulation Modes: Accumulating, Discarding, Accumulating & Retracting
ᵒ Triggers
• S torage
ᵒ In memory (checkpointed)
ᵒ Managed S tate
Stateful Windowed Processing
Compositional API
22
@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
23
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);
•Goal: low latency and high throughput
•Replicate processing logic, partition data/stream
•Configurable (Application.java or properties.xml)
•StreamCodec
•Controls 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 - S tatically 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
S calability - Partitioning
24
Scalability - Partitioning (contd.)
25
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
26
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.)
27
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.)
28
https://www.datatorrent.com/blog/fault-tolerant-file-processing/
Who is using Apex?
29
• 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-
using-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-
silver-spring-networks
Resources
30
• http://apex.apache.org/
• Learn more - http://apex.apache.org/docs.html
• Get involved - 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 S tartups -
https://www.datatorrent.com/product/startup-accelerator/
Q&A
31

Weitere ähnliche Inhalte

Was ist angesagt?

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 2016Gyula Fóra
 
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 StreamingApache Apex
 
From Batch to Streaming with Apache Apex Dataworks Summit 2017
From Batch to Streaming with Apache Apex Dataworks Summit 2017From Batch to Streaming with Apache Apex Dataworks Summit 2017
From Batch to Streaming with Apache Apex Dataworks Summit 2017Apache Apex
 
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 EcosystemGyula Fóra
 
Intro to Apache Apex @ Women in Big Data
Intro to Apache Apex @ Women in Big DataIntro to Apache Apex @ Women in Big Data
Intro to Apache Apex @ Women in Big DataApache 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
 
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache ApexApache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache ApexApache Apex
 
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
 
Spark Summit EU talk by Ram Sriharsha and Vlad Feinberg
Spark Summit EU talk by Ram Sriharsha and Vlad FeinbergSpark Summit EU talk by Ram Sriharsha and Vlad Feinberg
Spark Summit EU talk by Ram Sriharsha and Vlad FeinbergSpark Summit
 
Assaf Araki – Real Time Analytics at Scale
Assaf Araki – Real Time Analytics at ScaleAssaf Araki – Real Time Analytics at Scale
Assaf Araki – Real Time Analytics at ScaleFlink Forward
 
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
 
Unifying Stream, SWL and CEP for Declarative Stream Processing with Apache Flink
Unifying Stream, SWL and CEP for Declarative Stream Processing with Apache FlinkUnifying Stream, SWL and CEP for Declarative Stream Processing with Apache Flink
Unifying Stream, SWL and CEP for Declarative Stream Processing with Apache FlinkDataWorks Summit/Hadoop Summit
 
Introduction to Apache Apex
Introduction to Apache ApexIntroduction to Apache Apex
Introduction to Apache ApexApache Apex
 
Apache Spark vs Apache Flink
Apache Spark vs Apache FlinkApache Spark vs Apache Flink
Apache Spark vs Apache FlinkAKASH SIHAG
 
Apache con big data 2015 - Data Science from the trenches
Apache con big data 2015 - Data Science from the trenchesApache con big data 2015 - Data Science from the trenches
Apache con big data 2015 - Data Science from the trenchesVinay Shukla
 
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache FlinkAlbert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache FlinkFlink Forward
 
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
 
Extending The Yahoo Streaming Benchmark to Apache Apex
Extending The Yahoo Streaming Benchmark to Apache ApexExtending The Yahoo Streaming Benchmark to Apache Apex
Extending The Yahoo Streaming Benchmark to Apache ApexApache 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
 

Was ist angesagt? (20)

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
 
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
 
From Batch to Streaming with Apache Apex Dataworks Summit 2017
From Batch to Streaming with Apache Apex Dataworks Summit 2017From Batch to Streaming with Apache Apex Dataworks Summit 2017
From Batch to Streaming with Apache Apex Dataworks Summit 2017
 
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
 
Intro to Apache Apex @ Women in Big Data
Intro to Apache Apex @ Women in Big DataIntro to Apache Apex @ Women in Big Data
Intro to Apache Apex @ Women in Big Data
 
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
 
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache ApexApache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
Apache Big Data EU 2016: Next Gen Big Data Analytics with Apache Apex
 
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)
 
Spark Summit EU talk by Ram Sriharsha and Vlad Feinberg
Spark Summit EU talk by Ram Sriharsha and Vlad FeinbergSpark Summit EU talk by Ram Sriharsha and Vlad Feinberg
Spark Summit EU talk by Ram Sriharsha and Vlad Feinberg
 
Assaf Araki – Real Time Analytics at Scale
Assaf Araki – Real Time Analytics at ScaleAssaf Araki – Real Time Analytics at Scale
Assaf Araki – Real Time Analytics at Scale
 
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
 
Unifying Stream, SWL and CEP for Declarative Stream Processing with Apache Flink
Unifying Stream, SWL and CEP for Declarative Stream Processing with Apache FlinkUnifying Stream, SWL and CEP for Declarative Stream Processing with Apache Flink
Unifying Stream, SWL and CEP for Declarative Stream Processing with Apache Flink
 
Introduction to Apache Apex
Introduction to Apache ApexIntroduction to Apache Apex
Introduction to Apache Apex
 
Debunking Common Myths in Stream Processing
Debunking Common Myths in Stream ProcessingDebunking Common Myths in Stream Processing
Debunking Common Myths in Stream Processing
 
Apache Spark vs Apache Flink
Apache Spark vs Apache FlinkApache Spark vs Apache Flink
Apache Spark vs Apache Flink
 
Apache con big data 2015 - Data Science from the trenches
Apache con big data 2015 - Data Science from the trenchesApache con big data 2015 - Data Science from the trenches
Apache con big data 2015 - Data Science from the trenches
 
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache FlinkAlbert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
 
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
 
Extending The Yahoo Streaming Benchmark to Apache Apex
Extending The Yahoo Streaming Benchmark to Apache ApexExtending The Yahoo Streaming Benchmark to Apache Apex
Extending The Yahoo Streaming Benchmark to Apache Apex
 
Introduction to Real-Time Data Processing
Introduction to Real-Time Data ProcessingIntroduction to Real-Time Data Processing
Introduction to Real-Time Data Processing
 

Andere mochten auch

Case of success: Visualization as an example for exercising democratic transp...
Case of success: Visualization as an example for exercising democratic transp...Case of success: Visualization as an example for exercising democratic transp...
Case of success: Visualization as an example for exercising democratic transp...Big Data Spain
 
Big Migrations: Moving elephant herds by Carlos Izquierdo
Big Migrations: Moving elephant herds by Carlos IzquierdoBig Migrations: Moving elephant herds by Carlos Izquierdo
Big Migrations: Moving elephant herds by Carlos IzquierdoBig Data Spain
 
Assessing spatial accessibility to primary health care services in the Metrop...
Assessing spatial accessibility to primary health care services in the Metrop...Assessing spatial accessibility to primary health care services in the Metrop...
Assessing spatial accessibility to primary health care services in the Metrop...Big Data Spain
 
Advanced data science algorithms applied to scalable stream processing by Dav...
Advanced data science algorithms applied to scalable stream processing by Dav...Advanced data science algorithms applied to scalable stream processing by Dav...
Advanced data science algorithms applied to scalable stream processing by Dav...Big Data Spain
 
Enabling the Bank of the Future by Ignacio Bernal
Enabling the Bank of the Future by Ignacio BernalEnabling the Bank of the Future by Ignacio Bernal
Enabling the Bank of the Future by Ignacio BernalBig Data Spain
 
Finding the needle in the haystack: how Nestle is leveraging big data to defe...
Finding the needle in the haystack: how Nestle is leveraging big data to defe...Finding the needle in the haystack: how Nestle is leveraging big data to defe...
Finding the needle in the haystack: how Nestle is leveraging big data to defe...Big Data Spain
 
From data to AI with the Machine Learning Canvas by Louis Dorard Slides
From data to AI with the Machine Learning Canvas by Louis  Dorard SlidesFrom data to AI with the Machine Learning Canvas by Louis  Dorard Slides
From data to AI with the Machine Learning Canvas by Louis Dorard SlidesBig Data Spain
 
Converging Big Data and Application Infrastructure by Steven Poutsy
Converging Big Data and Application Infrastructure by Steven PoutsyConverging Big Data and Application Infrastructure by Steven Poutsy
Converging Big Data and Application Infrastructure by Steven PoutsyBig Data Spain
 
Big data in 140 characters by Joe Rice
Big data in 140 characters by Joe RiceBig data in 140 characters by Joe Rice
Big data in 140 characters by Joe RiceBig Data Spain
 
Are we reaching a Data Science Singularity? How Cognitive Computing is emergi...
Are we reaching a Data Science Singularity? How Cognitive Computing is emergi...Are we reaching a Data Science Singularity? How Cognitive Computing is emergi...
Are we reaching a Data Science Singularity? How Cognitive Computing is emergi...Big Data Spain
 
Why Apache Flink is better than Spark by Rubén Casado
Why Apache Flink is better than Spark by Rubén CasadoWhy Apache Flink is better than Spark by Rubén Casado
Why Apache Flink is better than Spark by Rubén CasadoBig Data Spain
 
VP of WW Partners by Alan Chhabra
VP of WW Partners by Alan ChhabraVP of WW Partners by Alan Chhabra
VP of WW Partners by Alan ChhabraBig Data Spain
 
From data to numbers to knowledge: semantic embeddings By Alvaro Barbero
From data to numbers to knowledge: semantic embeddings By Alvaro BarberoFrom data to numbers to knowledge: semantic embeddings By Alvaro Barbero
From data to numbers to knowledge: semantic embeddings By Alvaro BarberoBig Data Spain
 
Migration and Coexistence between Relational and NoSQL Databases by Manuel H...
 Migration and Coexistence between Relational and NoSQL Databases by Manuel H... Migration and Coexistence between Relational and NoSQL Databases by Manuel H...
Migration and Coexistence between Relational and NoSQL Databases by Manuel H...Big Data Spain
 
Top industry use cases for streaming analytics
Top industry use cases for streaming analyticsTop industry use cases for streaming analytics
Top industry use cases for streaming analyticsIBM Analytics
 
Apache Flink: Real-World Use Cases for Streaming Analytics
Apache Flink: Real-World Use Cases for Streaming AnalyticsApache Flink: Real-World Use Cases for Streaming Analytics
Apache Flink: Real-World Use Cases for Streaming AnalyticsSlim Baltagi
 
HOW TO APPLY BIG DATA ANALYTICS AND MACHINE LEARNING TO REAL TIME PROCESSING ...
HOW TO APPLY BIG DATA ANALYTICS AND MACHINE LEARNING TO REAL TIME PROCESSING ...HOW TO APPLY BIG DATA ANALYTICS AND MACHINE LEARNING TO REAL TIME PROCESSING ...
HOW TO APPLY BIG DATA ANALYTICS AND MACHINE LEARNING TO REAL TIME PROCESSING ...Big Data Spain
 
Introduction to Apache Apex by Thomas Weise
Introduction to Apache Apex by Thomas WeiseIntroduction to Apache Apex by Thomas Weise
Introduction to Apache Apex by Thomas WeiseBig Data Spain
 
DEPOTnext 23 February 2017- Orange Sky Laundry
DEPOTnext 23 February 2017- Orange Sky LaundryDEPOTnext 23 February 2017- Orange Sky Laundry
DEPOTnext 23 February 2017- Orange Sky Laundrybusiness_DEPOT
 
The Rise of Engineering-Driven Analytics by Loren Shure
The Rise of Engineering-Driven Analytics by Loren ShureThe Rise of Engineering-Driven Analytics by Loren Shure
The Rise of Engineering-Driven Analytics by Loren ShureBig Data Spain
 

Andere mochten auch (20)

Case of success: Visualization as an example for exercising democratic transp...
Case of success: Visualization as an example for exercising democratic transp...Case of success: Visualization as an example for exercising democratic transp...
Case of success: Visualization as an example for exercising democratic transp...
 
Big Migrations: Moving elephant herds by Carlos Izquierdo
Big Migrations: Moving elephant herds by Carlos IzquierdoBig Migrations: Moving elephant herds by Carlos Izquierdo
Big Migrations: Moving elephant herds by Carlos Izquierdo
 
Assessing spatial accessibility to primary health care services in the Metrop...
Assessing spatial accessibility to primary health care services in the Metrop...Assessing spatial accessibility to primary health care services in the Metrop...
Assessing spatial accessibility to primary health care services in the Metrop...
 
Advanced data science algorithms applied to scalable stream processing by Dav...
Advanced data science algorithms applied to scalable stream processing by Dav...Advanced data science algorithms applied to scalable stream processing by Dav...
Advanced data science algorithms applied to scalable stream processing by Dav...
 
Enabling the Bank of the Future by Ignacio Bernal
Enabling the Bank of the Future by Ignacio BernalEnabling the Bank of the Future by Ignacio Bernal
Enabling the Bank of the Future by Ignacio Bernal
 
Finding the needle in the haystack: how Nestle is leveraging big data to defe...
Finding the needle in the haystack: how Nestle is leveraging big data to defe...Finding the needle in the haystack: how Nestle is leveraging big data to defe...
Finding the needle in the haystack: how Nestle is leveraging big data to defe...
 
From data to AI with the Machine Learning Canvas by Louis Dorard Slides
From data to AI with the Machine Learning Canvas by Louis  Dorard SlidesFrom data to AI with the Machine Learning Canvas by Louis  Dorard Slides
From data to AI with the Machine Learning Canvas by Louis Dorard Slides
 
Converging Big Data and Application Infrastructure by Steven Poutsy
Converging Big Data and Application Infrastructure by Steven PoutsyConverging Big Data and Application Infrastructure by Steven Poutsy
Converging Big Data and Application Infrastructure by Steven Poutsy
 
Big data in 140 characters by Joe Rice
Big data in 140 characters by Joe RiceBig data in 140 characters by Joe Rice
Big data in 140 characters by Joe Rice
 
Are we reaching a Data Science Singularity? How Cognitive Computing is emergi...
Are we reaching a Data Science Singularity? How Cognitive Computing is emergi...Are we reaching a Data Science Singularity? How Cognitive Computing is emergi...
Are we reaching a Data Science Singularity? How Cognitive Computing is emergi...
 
Why Apache Flink is better than Spark by Rubén Casado
Why Apache Flink is better than Spark by Rubén CasadoWhy Apache Flink is better than Spark by Rubén Casado
Why Apache Flink is better than Spark by Rubén Casado
 
VP of WW Partners by Alan Chhabra
VP of WW Partners by Alan ChhabraVP of WW Partners by Alan Chhabra
VP of WW Partners by Alan Chhabra
 
From data to numbers to knowledge: semantic embeddings By Alvaro Barbero
From data to numbers to knowledge: semantic embeddings By Alvaro BarberoFrom data to numbers to knowledge: semantic embeddings By Alvaro Barbero
From data to numbers to knowledge: semantic embeddings By Alvaro Barbero
 
Migration and Coexistence between Relational and NoSQL Databases by Manuel H...
 Migration and Coexistence between Relational and NoSQL Databases by Manuel H... Migration and Coexistence between Relational and NoSQL Databases by Manuel H...
Migration and Coexistence between Relational and NoSQL Databases by Manuel H...
 
Top industry use cases for streaming analytics
Top industry use cases for streaming analyticsTop industry use cases for streaming analytics
Top industry use cases for streaming analytics
 
Apache Flink: Real-World Use Cases for Streaming Analytics
Apache Flink: Real-World Use Cases for Streaming AnalyticsApache Flink: Real-World Use Cases for Streaming Analytics
Apache Flink: Real-World Use Cases for Streaming Analytics
 
HOW TO APPLY BIG DATA ANALYTICS AND MACHINE LEARNING TO REAL TIME PROCESSING ...
HOW TO APPLY BIG DATA ANALYTICS AND MACHINE LEARNING TO REAL TIME PROCESSING ...HOW TO APPLY BIG DATA ANALYTICS AND MACHINE LEARNING TO REAL TIME PROCESSING ...
HOW TO APPLY BIG DATA ANALYTICS AND MACHINE LEARNING TO REAL TIME PROCESSING ...
 
Introduction to Apache Apex by Thomas Weise
Introduction to Apache Apex by Thomas WeiseIntroduction to Apache Apex by Thomas Weise
Introduction to Apache Apex by Thomas Weise
 
DEPOTnext 23 February 2017- Orange Sky Laundry
DEPOTnext 23 February 2017- Orange Sky LaundryDEPOTnext 23 February 2017- Orange Sky Laundry
DEPOTnext 23 February 2017- Orange Sky Laundry
 
The Rise of Engineering-Driven Analytics by Loren Shure
The Rise of Engineering-Driven Analytics by Loren ShureThe Rise of Engineering-Driven Analytics by Loren Shure
The Rise of Engineering-Driven Analytics by Loren Shure
 

Ähnlich wie Stream Processing use cases and applications with Apache Apex by Thomas Weise

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 ApexThomas Weise
 
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
 
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
 
Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...
Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...
Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...Provectus
 
Flink history, roadmap and vision
Flink history, roadmap and visionFlink history, roadmap and vision
Flink history, roadmap and visionStephan Ewen
 
Building Your First Apache Apex Application
Building Your First Apache Apex ApplicationBuilding Your First Apache Apex Application
Building Your First Apache Apex ApplicationApache Apex
 
Python Streaming Pipelines on Flink - Beam Meetup at Lyft 2019
Python Streaming Pipelines on Flink - Beam Meetup at Lyft 2019Python Streaming Pipelines on Flink - Beam Meetup at Lyft 2019
Python Streaming Pipelines on Flink - Beam Meetup at Lyft 2019Thomas Weise
 
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
 
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 DevelopmentApache Apex
 
開放原始碼 Ch1.2 intro - oss - apahce foundry (ver 2.0)
開放原始碼 Ch1.2   intro - oss - apahce foundry (ver 2.0)開放原始碼 Ch1.2   intro - oss - apahce foundry (ver 2.0)
開放原始碼 Ch1.2 intro - oss - apahce foundry (ver 2.0)My own sweet home!
 
Practices and tools for building better APIs
Practices and tools for building better APIsPractices and tools for building better APIs
Practices and tools for building better APIsNLJUG
 
Practices and tools for building better API (JFall 2013)
Practices and tools for building better API (JFall 2013)Practices and tools for building better API (JFall 2013)
Practices and tools for building better API (JFall 2013)Peter Hendriks
 
Scorex, the Modular Blockchain Framework
Scorex, the Modular Blockchain FrameworkScorex, the Modular Blockchain Framework
Scorex, the Modular Blockchain FrameworkAlex Chepurnoy
 
Apache Beam @ GCPUG.TW Flink.TW 20161006
Apache Beam @ GCPUG.TW Flink.TW 20161006Apache Beam @ GCPUG.TW Flink.TW 20161006
Apache Beam @ GCPUG.TW Flink.TW 20161006Randy Huang
 
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...Guido Schmutz
 
Portable Streaming Pipelines with Apache Beam
Portable Streaming Pipelines with Apache BeamPortable Streaming Pipelines with Apache Beam
Portable Streaming Pipelines with Apache Beamconfluent
 
Flink Forward SF 2017: Malo Deniélou - No shard left behind: Dynamic work re...
Flink Forward SF 2017: Malo Deniélou -  No shard left behind: Dynamic work re...Flink Forward SF 2017: Malo Deniélou -  No shard left behind: Dynamic work re...
Flink Forward SF 2017: Malo Deniélou - No shard left behind: Dynamic work re...Flink Forward
 
Real-Time Log Analysis with Apache Mesos, Kafka and Cassandra
Real-Time Log Analysis with Apache Mesos, Kafka and CassandraReal-Time Log Analysis with Apache Mesos, Kafka and Cassandra
Real-Time Log Analysis with Apache Mesos, Kafka and CassandraJoe Stein
 

Ähnlich wie Stream Processing use cases and applications with Apache Apex by Thomas Weise (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
 
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
 
Introduction to Apache Apex
Introduction to Apache ApexIntroduction to Apache Apex
Introduction to Apache 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)
 
Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...
Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...
Data Summer Conf 2018, “Building unified Batch and Stream processing pipeline...
 
Flink history, roadmap and vision
Flink history, roadmap and visionFlink history, roadmap and vision
Flink history, roadmap and vision
 
Building Your First Apache Apex Application
Building Your First Apache Apex ApplicationBuilding Your First Apache Apex Application
Building Your First Apache Apex Application
 
Python Streaming Pipelines on Flink - Beam Meetup at Lyft 2019
Python Streaming Pipelines on Flink - Beam Meetup at Lyft 2019Python Streaming Pipelines on Flink - Beam Meetup at Lyft 2019
Python Streaming Pipelines on Flink - Beam Meetup at Lyft 2019
 
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 Beam
Introduction to Apache BeamIntroduction to Apache Beam
Introduction to Apache Beam
 
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
 
開放原始碼 Ch1.2 intro - oss - apahce foundry (ver 2.0)
開放原始碼 Ch1.2   intro - oss - apahce foundry (ver 2.0)開放原始碼 Ch1.2   intro - oss - apahce foundry (ver 2.0)
開放原始碼 Ch1.2 intro - oss - apahce foundry (ver 2.0)
 
Practices and tools for building better APIs
Practices and tools for building better APIsPractices and tools for building better APIs
Practices and tools for building better APIs
 
Practices and tools for building better API (JFall 2013)
Practices and tools for building better API (JFall 2013)Practices and tools for building better API (JFall 2013)
Practices and tools for building better API (JFall 2013)
 
Scorex, the Modular Blockchain Framework
Scorex, the Modular Blockchain FrameworkScorex, the Modular Blockchain Framework
Scorex, the Modular Blockchain Framework
 
Apache Beam @ GCPUG.TW Flink.TW 20161006
Apache Beam @ GCPUG.TW Flink.TW 20161006Apache Beam @ GCPUG.TW Flink.TW 20161006
Apache Beam @ GCPUG.TW Flink.TW 20161006
 
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...
 
Portable Streaming Pipelines with Apache Beam
Portable Streaming Pipelines with Apache BeamPortable Streaming Pipelines with Apache Beam
Portable Streaming Pipelines with Apache Beam
 
Flink Forward SF 2017: Malo Deniélou - No shard left behind: Dynamic work re...
Flink Forward SF 2017: Malo Deniélou -  No shard left behind: Dynamic work re...Flink Forward SF 2017: Malo Deniélou -  No shard left behind: Dynamic work re...
Flink Forward SF 2017: Malo Deniélou - No shard left behind: Dynamic work re...
 
Real-Time Log Analysis with Apache Mesos, Kafka and Cassandra
Real-Time Log Analysis with Apache Mesos, Kafka and CassandraReal-Time Log Analysis with Apache Mesos, Kafka and Cassandra
Real-Time Log Analysis with Apache Mesos, Kafka and Cassandra
 

Mehr von Big Data Spain

Big Data, Big Quality? by Irene Gonzálvez at Big Data Spain 2017
Big Data, Big Quality? by Irene Gonzálvez at Big Data Spain 2017Big Data, Big Quality? by Irene Gonzálvez at Big Data Spain 2017
Big Data, Big Quality? by Irene Gonzálvez at Big Data Spain 2017Big Data Spain
 
Scaling a backend for a big data and blockchain environment by Rafael Ríos at...
Scaling a backend for a big data and blockchain environment by Rafael Ríos at...Scaling a backend for a big data and blockchain environment by Rafael Ríos at...
Scaling a backend for a big data and blockchain environment by Rafael Ríos at...Big Data Spain
 
AI: The next frontier by Amparo Alonso at Big Data Spain 2017
AI: The next frontier by Amparo Alonso at Big Data Spain 2017AI: The next frontier by Amparo Alonso at Big Data Spain 2017
AI: The next frontier by Amparo Alonso at Big Data Spain 2017Big Data Spain
 
Disaster Recovery for Big Data by Carlos Izquierdo at Big Data Spain 2017
Disaster Recovery for Big Data by Carlos Izquierdo at Big Data Spain 2017Disaster Recovery for Big Data by Carlos Izquierdo at Big Data Spain 2017
Disaster Recovery for Big Data by Carlos Izquierdo at Big Data Spain 2017Big Data Spain
 
Presentation: Boost Hadoop and Spark with in-memory technologies by Akmal Cha...
Presentation: Boost Hadoop and Spark with in-memory technologies by Akmal Cha...Presentation: Boost Hadoop and Spark with in-memory technologies by Akmal Cha...
Presentation: Boost Hadoop and Spark with in-memory technologies by Akmal Cha...Big Data Spain
 
Data science for lazy people, Automated Machine Learning by Diego Hueltes at ...
Data science for lazy people, Automated Machine Learning by Diego Hueltes at ...Data science for lazy people, Automated Machine Learning by Diego Hueltes at ...
Data science for lazy people, Automated Machine Learning by Diego Hueltes at ...Big Data Spain
 
Training Deep Learning Models on Multiple GPUs in the Cloud by Enrique Otero ...
Training Deep Learning Models on Multiple GPUs in the Cloud by Enrique Otero ...Training Deep Learning Models on Multiple GPUs in the Cloud by Enrique Otero ...
Training Deep Learning Models on Multiple GPUs in the Cloud by Enrique Otero ...Big Data Spain
 
Unbalanced data: Same algorithms different techniques by Eric Martín at Big D...
Unbalanced data: Same algorithms different techniques by Eric Martín at Big D...Unbalanced data: Same algorithms different techniques by Eric Martín at Big D...
Unbalanced data: Same algorithms different techniques by Eric Martín at Big D...Big Data Spain
 
State of the art time-series analysis with deep learning by Javier Ordóñez at...
State of the art time-series analysis with deep learning by Javier Ordóñez at...State of the art time-series analysis with deep learning by Javier Ordóñez at...
State of the art time-series analysis with deep learning by Javier Ordóñez at...Big Data Spain
 
Trading at market speed with the latest Kafka features by Iñigo González at B...
Trading at market speed with the latest Kafka features by Iñigo González at B...Trading at market speed with the latest Kafka features by Iñigo González at B...
Trading at market speed with the latest Kafka features by Iñigo González at B...Big Data Spain
 
Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...
Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...
Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...Big Data Spain
 
The Analytic Platform behind IBM’s Watson Data Platform by Luciano Resende a...
 The Analytic Platform behind IBM’s Watson Data Platform by Luciano Resende a... The Analytic Platform behind IBM’s Watson Data Platform by Luciano Resende a...
The Analytic Platform behind IBM’s Watson Data Platform by Luciano Resende a...Big Data Spain
 
Artificial Intelligence and Data-centric businesses by Óscar Méndez at Big Da...
Artificial Intelligence and Data-centric businesses by Óscar Méndez at Big Da...Artificial Intelligence and Data-centric businesses by Óscar Méndez at Big Da...
Artificial Intelligence and Data-centric businesses by Óscar Méndez at Big Da...Big Data Spain
 
Why big data didn’t end causal inference by Totte Harinen at Big Data Spain 2017
Why big data didn’t end causal inference by Totte Harinen at Big Data Spain 2017Why big data didn’t end causal inference by Totte Harinen at Big Data Spain 2017
Why big data didn’t end causal inference by Totte Harinen at Big Data Spain 2017Big Data Spain
 
Meme Index. Analyzing fads and sensations on the Internet by Miguel Romero at...
Meme Index. Analyzing fads and sensations on the Internet by Miguel Romero at...Meme Index. Analyzing fads and sensations on the Internet by Miguel Romero at...
Meme Index. Analyzing fads and sensations on the Internet by Miguel Romero at...Big Data Spain
 
Vehicle Big Data that Drives Smart City Advancement by Mike Branch at Big Dat...
Vehicle Big Data that Drives Smart City Advancement by Mike Branch at Big Dat...Vehicle Big Data that Drives Smart City Advancement by Mike Branch at Big Dat...
Vehicle Big Data that Drives Smart City Advancement by Mike Branch at Big Dat...Big Data Spain
 
End of the Myth: Ultra-Scalable Transactional Management by Ricardo Jiménez-P...
End of the Myth: Ultra-Scalable Transactional Management by Ricardo Jiménez-P...End of the Myth: Ultra-Scalable Transactional Management by Ricardo Jiménez-P...
End of the Myth: Ultra-Scalable Transactional Management by Ricardo Jiménez-P...Big Data Spain
 
Attacking Machine Learning used in AntiVirus with Reinforcement by Rubén Mart...
Attacking Machine Learning used in AntiVirus with Reinforcement by Rubén Mart...Attacking Machine Learning used in AntiVirus with Reinforcement by Rubén Mart...
Attacking Machine Learning used in AntiVirus with Reinforcement by Rubén Mart...Big Data Spain
 
More people, less banking: Blockchain by Salvador Casquero at Big Data Spain ...
More people, less banking: Blockchain by Salvador Casquero at Big Data Spain ...More people, less banking: Blockchain by Salvador Casquero at Big Data Spain ...
More people, less banking: Blockchain by Salvador Casquero at Big Data Spain ...Big Data Spain
 
Make the elephant fly, once again by Sourygna Luangsay at Big Data Spain 2017
Make the elephant fly, once again by Sourygna Luangsay at Big Data Spain 2017Make the elephant fly, once again by Sourygna Luangsay at Big Data Spain 2017
Make the elephant fly, once again by Sourygna Luangsay at Big Data Spain 2017Big Data Spain
 

Mehr von Big Data Spain (20)

Big Data, Big Quality? by Irene Gonzálvez at Big Data Spain 2017
Big Data, Big Quality? by Irene Gonzálvez at Big Data Spain 2017Big Data, Big Quality? by Irene Gonzálvez at Big Data Spain 2017
Big Data, Big Quality? by Irene Gonzálvez at Big Data Spain 2017
 
Scaling a backend for a big data and blockchain environment by Rafael Ríos at...
Scaling a backend for a big data and blockchain environment by Rafael Ríos at...Scaling a backend for a big data and blockchain environment by Rafael Ríos at...
Scaling a backend for a big data and blockchain environment by Rafael Ríos at...
 
AI: The next frontier by Amparo Alonso at Big Data Spain 2017
AI: The next frontier by Amparo Alonso at Big Data Spain 2017AI: The next frontier by Amparo Alonso at Big Data Spain 2017
AI: The next frontier by Amparo Alonso at Big Data Spain 2017
 
Disaster Recovery for Big Data by Carlos Izquierdo at Big Data Spain 2017
Disaster Recovery for Big Data by Carlos Izquierdo at Big Data Spain 2017Disaster Recovery for Big Data by Carlos Izquierdo at Big Data Spain 2017
Disaster Recovery for Big Data by Carlos Izquierdo at Big Data Spain 2017
 
Presentation: Boost Hadoop and Spark with in-memory technologies by Akmal Cha...
Presentation: Boost Hadoop and Spark with in-memory technologies by Akmal Cha...Presentation: Boost Hadoop and Spark with in-memory technologies by Akmal Cha...
Presentation: Boost Hadoop and Spark with in-memory technologies by Akmal Cha...
 
Data science for lazy people, Automated Machine Learning by Diego Hueltes at ...
Data science for lazy people, Automated Machine Learning by Diego Hueltes at ...Data science for lazy people, Automated Machine Learning by Diego Hueltes at ...
Data science for lazy people, Automated Machine Learning by Diego Hueltes at ...
 
Training Deep Learning Models on Multiple GPUs in the Cloud by Enrique Otero ...
Training Deep Learning Models on Multiple GPUs in the Cloud by Enrique Otero ...Training Deep Learning Models on Multiple GPUs in the Cloud by Enrique Otero ...
Training Deep Learning Models on Multiple GPUs in the Cloud by Enrique Otero ...
 
Unbalanced data: Same algorithms different techniques by Eric Martín at Big D...
Unbalanced data: Same algorithms different techniques by Eric Martín at Big D...Unbalanced data: Same algorithms different techniques by Eric Martín at Big D...
Unbalanced data: Same algorithms different techniques by Eric Martín at Big D...
 
State of the art time-series analysis with deep learning by Javier Ordóñez at...
State of the art time-series analysis with deep learning by Javier Ordóñez at...State of the art time-series analysis with deep learning by Javier Ordóñez at...
State of the art time-series analysis with deep learning by Javier Ordóñez at...
 
Trading at market speed with the latest Kafka features by Iñigo González at B...
Trading at market speed with the latest Kafka features by Iñigo González at B...Trading at market speed with the latest Kafka features by Iñigo González at B...
Trading at market speed with the latest Kafka features by Iñigo González at B...
 
Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...
Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...
Unified Stream Processing at Scale with Apache Samza by Jake Maes at Big Data...
 
The Analytic Platform behind IBM’s Watson Data Platform by Luciano Resende a...
 The Analytic Platform behind IBM’s Watson Data Platform by Luciano Resende a... The Analytic Platform behind IBM’s Watson Data Platform by Luciano Resende a...
The Analytic Platform behind IBM’s Watson Data Platform by Luciano Resende a...
 
Artificial Intelligence and Data-centric businesses by Óscar Méndez at Big Da...
Artificial Intelligence and Data-centric businesses by Óscar Méndez at Big Da...Artificial Intelligence and Data-centric businesses by Óscar Méndez at Big Da...
Artificial Intelligence and Data-centric businesses by Óscar Méndez at Big Da...
 
Why big data didn’t end causal inference by Totte Harinen at Big Data Spain 2017
Why big data didn’t end causal inference by Totte Harinen at Big Data Spain 2017Why big data didn’t end causal inference by Totte Harinen at Big Data Spain 2017
Why big data didn’t end causal inference by Totte Harinen at Big Data Spain 2017
 
Meme Index. Analyzing fads and sensations on the Internet by Miguel Romero at...
Meme Index. Analyzing fads and sensations on the Internet by Miguel Romero at...Meme Index. Analyzing fads and sensations on the Internet by Miguel Romero at...
Meme Index. Analyzing fads and sensations on the Internet by Miguel Romero at...
 
Vehicle Big Data that Drives Smart City Advancement by Mike Branch at Big Dat...
Vehicle Big Data that Drives Smart City Advancement by Mike Branch at Big Dat...Vehicle Big Data that Drives Smart City Advancement by Mike Branch at Big Dat...
Vehicle Big Data that Drives Smart City Advancement by Mike Branch at Big Dat...
 
End of the Myth: Ultra-Scalable Transactional Management by Ricardo Jiménez-P...
End of the Myth: Ultra-Scalable Transactional Management by Ricardo Jiménez-P...End of the Myth: Ultra-Scalable Transactional Management by Ricardo Jiménez-P...
End of the Myth: Ultra-Scalable Transactional Management by Ricardo Jiménez-P...
 
Attacking Machine Learning used in AntiVirus with Reinforcement by Rubén Mart...
Attacking Machine Learning used in AntiVirus with Reinforcement by Rubén Mart...Attacking Machine Learning used in AntiVirus with Reinforcement by Rubén Mart...
Attacking Machine Learning used in AntiVirus with Reinforcement by Rubén Mart...
 
More people, less banking: Blockchain by Salvador Casquero at Big Data Spain ...
More people, less banking: Blockchain by Salvador Casquero at Big Data Spain ...More people, less banking: Blockchain by Salvador Casquero at Big Data Spain ...
More people, less banking: Blockchain by Salvador Casquero at Big Data Spain ...
 
Make the elephant fly, once again by Sourygna Luangsay at Big Data Spain 2017
Make the elephant fly, once again by Sourygna Luangsay at Big Data Spain 2017Make the elephant fly, once again by Sourygna Luangsay at Big Data Spain 2017
Make the elephant fly, once again by Sourygna Luangsay at Big Data Spain 2017
 

Kürzlich hochgeladen

Why Teams call analytics are critical to your entire business
Why Teams call analytics are critical to your entire businessWhy Teams call analytics are critical to your entire business
Why Teams call analytics are critical to your entire businesspanagenda
 
2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...Martijn de Jong
 
Manulife - Insurer Transformation Award 2024
Manulife - Insurer Transformation Award 2024Manulife - Insurer Transformation Award 2024
Manulife - Insurer Transformation Award 2024The Digital Insurer
 
presentation ICT roal in 21st century education
presentation ICT roal in 21st century educationpresentation ICT roal in 21st century education
presentation ICT roal in 21st century educationjfdjdjcjdnsjd
 
"I see eyes in my soup": How Delivery Hero implemented the safety system for ...
"I see eyes in my soup": How Delivery Hero implemented the safety system for ..."I see eyes in my soup": How Delivery Hero implemented the safety system for ...
"I see eyes in my soup": How Delivery Hero implemented the safety system for ...Zilliz
 
[BuildWithAI] Introduction to Gemini.pdf
[BuildWithAI] Introduction to Gemini.pdf[BuildWithAI] Introduction to Gemini.pdf
[BuildWithAI] Introduction to Gemini.pdfSandro Moreira
 
DEV meet-up UiPath Document Understanding May 7 2024 Amsterdam
DEV meet-up UiPath Document Understanding May 7 2024 AmsterdamDEV meet-up UiPath Document Understanding May 7 2024 Amsterdam
DEV meet-up UiPath Document Understanding May 7 2024 AmsterdamUiPathCommunity
 
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...apidays
 
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc
 
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...Angeliki Cooney
 
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdfRising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdfOrbitshub
 
MS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectorsMS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectorsNanddeep Nachan
 
EMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWER
EMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWEREMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWER
EMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWERMadyBayot
 
Boost Fertility New Invention Ups Success Rates.pdf
Boost Fertility New Invention Ups Success Rates.pdfBoost Fertility New Invention Ups Success Rates.pdf
Boost Fertility New Invention Ups Success Rates.pdfsudhanshuwaghmare1
 
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers:  A Deep Dive into Serverless Spatial Data and FMECloud Frontiers:  A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FMESafe Software
 
Ransomware_Q4_2023. The report. [EN].pdf
Ransomware_Q4_2023. The report. [EN].pdfRansomware_Q4_2023. The report. [EN].pdf
Ransomware_Q4_2023. The report. [EN].pdfOverkill Security
 
AWS Community Day CPH - Three problems of Terraform
AWS Community Day CPH - Three problems of TerraformAWS Community Day CPH - Three problems of Terraform
AWS Community Day CPH - Three problems of TerraformAndrey Devyatkin
 
Corporate and higher education May webinar.pptx
Corporate and higher education May webinar.pptxCorporate and higher education May webinar.pptx
Corporate and higher education May webinar.pptxRustici Software
 
FWD Group - Insurer Innovation Award 2024
FWD Group - Insurer Innovation Award 2024FWD Group - Insurer Innovation Award 2024
FWD Group - Insurer Innovation Award 2024The Digital Insurer
 
Strategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
Strategize a Smooth Tenant-to-tenant Migration and Copilot TakeoffStrategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
Strategize a Smooth Tenant-to-tenant Migration and Copilot Takeoffsammart93
 

Kürzlich hochgeladen (20)

Why Teams call analytics are critical to your entire business
Why Teams call analytics are critical to your entire businessWhy Teams call analytics are critical to your entire business
Why Teams call analytics are critical to your entire business
 
2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...
 
Manulife - Insurer Transformation Award 2024
Manulife - Insurer Transformation Award 2024Manulife - Insurer Transformation Award 2024
Manulife - Insurer Transformation Award 2024
 
presentation ICT roal in 21st century education
presentation ICT roal in 21st century educationpresentation ICT roal in 21st century education
presentation ICT roal in 21st century education
 
"I see eyes in my soup": How Delivery Hero implemented the safety system for ...
"I see eyes in my soup": How Delivery Hero implemented the safety system for ..."I see eyes in my soup": How Delivery Hero implemented the safety system for ...
"I see eyes in my soup": How Delivery Hero implemented the safety system for ...
 
[BuildWithAI] Introduction to Gemini.pdf
[BuildWithAI] Introduction to Gemini.pdf[BuildWithAI] Introduction to Gemini.pdf
[BuildWithAI] Introduction to Gemini.pdf
 
DEV meet-up UiPath Document Understanding May 7 2024 Amsterdam
DEV meet-up UiPath Document Understanding May 7 2024 AmsterdamDEV meet-up UiPath Document Understanding May 7 2024 Amsterdam
DEV meet-up UiPath Document Understanding May 7 2024 Amsterdam
 
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
Apidays New York 2024 - Passkeys: Developing APIs to enable passwordless auth...
 
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
 
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
Biography Of Angeliki Cooney | Senior Vice President Life Sciences | Albany, ...
 
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdfRising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
Rising Above_ Dubai Floods and the Fortitude of Dubai International Airport.pdf
 
MS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectorsMS Copilot expands with MS Graph connectors
MS Copilot expands with MS Graph connectors
 
EMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWER
EMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWEREMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWER
EMPOWERMENT TECHNOLOGY GRADE 11 QUARTER 2 REVIEWER
 
Boost Fertility New Invention Ups Success Rates.pdf
Boost Fertility New Invention Ups Success Rates.pdfBoost Fertility New Invention Ups Success Rates.pdf
Boost Fertility New Invention Ups Success Rates.pdf
 
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers:  A Deep Dive into Serverless Spatial Data and FMECloud Frontiers:  A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
 
Ransomware_Q4_2023. The report. [EN].pdf
Ransomware_Q4_2023. The report. [EN].pdfRansomware_Q4_2023. The report. [EN].pdf
Ransomware_Q4_2023. The report. [EN].pdf
 
AWS Community Day CPH - Three problems of Terraform
AWS Community Day CPH - Three problems of TerraformAWS Community Day CPH - Three problems of Terraform
AWS Community Day CPH - Three problems of Terraform
 
Corporate and higher education May webinar.pptx
Corporate and higher education May webinar.pptxCorporate and higher education May webinar.pptx
Corporate and higher education May webinar.pptx
 
FWD Group - Insurer Innovation Award 2024
FWD Group - Insurer Innovation Award 2024FWD Group - Insurer Innovation Award 2024
FWD Group - Insurer Innovation Award 2024
 
Strategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
Strategize a Smooth Tenant-to-tenant Migration and Copilot TakeoffStrategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
Strategize a Smooth Tenant-to-tenant Migration and Copilot Takeoff
 

Stream Processing use cases and applications with Apache Apex by Thomas Weise

  • 1. 1
  • 2. Building Streaming Applications with Apache Apex Thomas Weise <thw@apache.org> @thweise PMC Chair Apache Apex, Architect DataTorrent Big Data Spain, Madrid, Nov 18th 2016
  • 3. Agenda 3 • Application Development Model • Creating Apex Application - Project S tructure • Apex APIs • Configuration Example • Operator APIs • Overview of Operator Library • Frequently used Connectors • S tateful Transformation & Windowing • S calability – Partitioning • End-to-end Exactly Once
  • 4. Application Development Model 4 ▪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) Output Stream Tupl e Tupl e er Operator er Operator er Operator er Operator er Operator er Operator
  • 5. Creating Apex Application Project 5 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
  • 6. Apex Application Project Structure 6 • pom.xml •Defines project structure and dependencies •Application.java •Defines the DAG •RandomNumberGenerator.java •S ample Operator •properties.xml •Contains operator and application properties and attributes •ApplicationTest.java •S ample test to test application in local mode
  • 7. APIs: Compositional (Low level) 7 Input Parser Counter Output CountsWordsLines Kafka Database Filter Filtered
  • 8. Apex APIs: Declarative (High Level) 8 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);
  • 9. APIs: S QL 9 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%'");
  • 10. APIs: Beam 10 • Apex Runner for Apache Beam is now available!! •Build once run-anywhere model •Beam S treaming 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(); }
  • 11. APIs: S AMOA 11 •Build once run-anywhere model for online machine learning algorithms •Any machine learning algorithm present in S AMOA can be run directly on Apex. •Uses Apex Iteration S upport •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)"
  • 12. Configuration (properties.xml) 12 Input Parser Counter Output CountsWordsLines Kafka Database Filter Filtered
  • 13. Streaming Window Processing Time Window 13 •Finite time sliced windows based on processing (event arrival) time •Used for bookkeeping of streaming application •Derived Windows are: Checkpoint Windows, Committed Windows
  • 14. Operator APIs 14 Next streaming window Next streaming window Input Adapters - S tarting 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()
  • 15. Operator Library 15 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
  • 16. Frequently used Connectors Kafka Input 16 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
  • 17. Frequently used Connectors Kafka Output 17 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
  • 18. Frequently used Connectors File Input 18 •AbstractFileInputOperator •Used to read a file from source and emit the content of the file to downstream operator •Operator is idempotent •S upports Partitioning •Few Concrete Impl •FileLineInputOperator •AvroFileInputOperator •ParquetFilePOJ OReader •https://www.datatorrent.com/blog/f ault-tolerant-file-processing/
  • 19. Frequently used Connectors File Output 19 •AbstractFileOutputOperator •Writes data to a file •S upports Partitions •Exactly-once results •Upstream operators should be idempotent •Few Concrete Impl •S tringFileOutputOperator •https://www.datatorrent.com/blog/f ault-tolerant-file-processing/
  • 20. Windowing Support 20 • Event-time Windows • Computation based on event-time present in the tuple • Types of event-time windows: ᵒ Global : S ingle 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 predefined min gap
  • 21. Stateful Windowed Processing 21 • WindowedOperator (org.apache.apex:malhar-library) • Used to process data based on Event time as contrary to ingression time • S upports windowing semantics of Apache Beam model • Features: ᵒ Watermarks ᵒ Allowed Lateness ᵒ Accumulation ᵒ Accumulation Modes: Accumulating, Discarding, Accumulating & Retracting ᵒ Triggers • S torage ᵒ In memory (checkpointed) ᵒ Managed S tate
  • 22. Stateful Windowed Processing Compositional API 22 @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); }
  • 23. Stateful Windowed Processing Declarative API 23 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);
  • 24. •Goal: low latency and high throughput •Replicate processing logic, partition data/stream •Configurable (Application.java or properties.xml) •StreamCodec •Controls 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 - S tatically 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 S calability - Partitioning 24
  • 25. Scalability - Partitioning (contd.) 25 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>
  • 26. End-to-End Exactly-Once 26 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/
  • 27. End-to-End Exactly-Once (contd.) 27 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()); } }
  • 29. Who is using Apex? 29 • 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- using-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- silver-spring-networks
  • 30. Resources 30 • http://apex.apache.org/ • Learn more - http://apex.apache.org/docs.html • Get involved - 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 S tartups - https://www.datatorrent.com/product/startup-accelerator/