SlideShare ist ein Scribd-Unternehmen logo
1 von 50
Downloaden Sie, um offline zu lesen
Spark vs Flink
Rumble in the (Big Data) Jungle
,
München, 2016-04-20
Konstantin Knauf Michael Pisula
Background
The Big Data Ecosystem
Apache Top-Level Projects over Time
2008 2010 2013 2014 2015
The New Guard
Berkeley University Origin TU Berlin
2013 Apache
Incubator
04/2014
02/2014 Apache Top-
Level
01/2015
databricks Company data Artisans
Scala, Java, Python, R Supported
languages
Java, Scala, Python
Scala Implemented
in
Java
Stand-Alone, Mesos,
EC2, YARN
Cluster Stand-Alone, Mesos, EC2, YARN
Lightning-fast cluster
computing
Teaser Scalable Batch and Stream
Data Processing
The Challenge
Real-Time Analysis of a Superhero Fight Club
Fight
hitter: Int
hittee: Int
hitpoints: Int
Segment
id: Int
name: String
segment: String
Detail
name: String
gender: Int
birthYear: Int
noOfAppearances: Int
Fight
hitter: Int
hittee: Int
hitpoints: Int
Fight
hitter: Int
hittee: Int
hitpoints: Int
Fight
hitter: Int
hittee: Int
hitpoints: Int
Fight
hitter: Int
hittee: Int
hitpoints: Int
Fight
hitter: Int
hittee: Int
hitpoints: Int
Fight
hitter: Int
hittee: Int
hitpoints: Int
Fight
hitter: Int
hittee: Int
hitpoints: Int
Hero
id: Int
name: String
segment: String
gender: Int
birthYear: Int
noOfAppearances: Int
{Stream
{Batch
The Setup
AWS Cluster
Kafka
Cluster
Stream ProcessingBatch Processing
Heroes
Combining Stream and Batch
Segment Detail Data Generator
Avro
Avro
Round 1
Setting up
Dependencies
compile "org.apache.flink:flink-java:1.0.0"
compile "org.apache.flink:flink-streaming-java_2.11:1.0.0"
//For Local Execution from IDE
compile "org.apache.flink:flink-clients_2.11:1.0.0"
Skeleton
//Batch (DataSetAPI)
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
//Stream (DataStream API)
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment()
//Processing Logic
//For Streaming
env.execute()
Dependencies
compile 'org.apache.spark:spark-core_2.10:1.5.0'
compile 'org.apache.spark:spark-streaming_2.10:1.5.0'
Skeleton Batch
SparkConf conf = new SparkConf().setAppName(appName).setMaster(master);
// Batch
JavaSparkContext sparkContext = new JavaSparkContext(conf);
// Stream
JavaStreamingContext jssc = new JavaStreamingContext(conf,
Durations.seconds(1));
// Processing Logic
jssc.start(); // For Streaming
First Impressions
Practically no boiler plate
Easy to get started and play around
Runs in the IDE
Hadoop MapReduce is much harder to get into
Round 2
Static Data Analysis
Combine both static data parts
Read the csv file and transform it
JavaRDD<String> segmentFile = sparkContext.textFile("s3://...");
JavaPairRDD<Integer, SegmentTableRecord> segmentTable = segmentFile
.map(line -> line.split(","))
.filter(array -> array.length == 3)
.mapToPair((String[] parts) -> {
int id = Integer.parseInt(parts[0]);
String name = parts[1], segment = parts[2];
return new Tuple2<>(name, new SegmentTableRecord(id, name, segment));
});
Join with detail data, filter out humans and write output
segmentTable.join(detailTable)
.mapValues(tuple -> {
SegmentTableRecord s = tuple._1();
DetailTableRecord d = tuple._2();
return new Hero(s.getId(), s.getName(), s.getSegment(),
d.getGender(), d.getBirthYear(), d.getNoOfAppearances());
})
.map(tuple -> tuple._2())
.filter(hero -> hero.getSegment().equals(HUMAN_SEGMENT))
.saveAsTextFile("s3://...");
Loading Files from S3 into POJO
DataSource<SegmentTableRecord> segmentTable = env.readCsvFile("s3://...")
.ignoreInvalidLines()
.pojoType(SegmentTableRecord.class, "id", "name", "segment");
Join and Filter
DataSet<Hero> humanHeros = segmentTable.join(detailTable)
.where("name")
.equalTo("name")
.with((s, d) -> new Hero(s.id, s.name, s.segment,
d.gender, d.birthYear, d.noOfAppearances))
.filter(hero -> hero.segment.equals("Human"));
Write back to S3
humanHeros.writeAsFormattedText(outputTablePath, WriteMode.OVERWRITE,
h -> h.toCsv());
Performance
Terasort1: Flink ca 66% of runtime
Terasort2: Flink ca. 68% of runtime
HashJoin: Flink ca. 32% of runtime
(Iterative Processes: Flink ca. 50% of runtime, ca. 7% with
Delta-Iterations)
2nd Round Points
Generally similar abstraction and feature set
Flink has a nicer syntax, more sugar
Spark is pretty bare-metal
Flink is faster
Round 3
Simple Real Time Analysis
Total Hitpoints over Last Minute
Configuring Environment for EventTime
StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
ExecutionConfig config = env.getConfig();
config.setAutoWatermarkInterval(500);
Creating Stream from Kafka
Properties properties = new Properties();
properties.put("bootstrap.servers", KAFKA_BROKERS);
properties.put("zookeeper.connect", ZOOKEEPER_CONNECTION);
properties.put("group.id", KAFKA_GROUP_ID);
DataStreamSource<FightEvent> hitStream =
env.addSource(new FlinkKafkaConsumer08<>("FightEventTopic",
new FightEventDeserializer(),
properties));
Processing Logic
hitStream.assignTimestamps(new FightEventTimestampExtractor(6000))
.timeWindowAll(Time.of(60, TimeUnit.SECONDS),
Time.of(10, TimeUnit.SECONDS))
.apply(new SumAllWindowFunction<FightEvent>() {
@Override
public long getSummand(FightEvent fightEvent) {
return fightEvent.getHitPoints();
}
})
.writeAsCsv("s3://...");
Example Output
3> (1448130670000,1448130730000,290789)
4> (1448130680000,1448130740000,289395)
5> (1448130690000,1448130750000,291768)
6> (1448130700000,1448130760000,292634)
7> (1448130710000,1448130770000,293869)
8> (1448130720000,1448130780000,293356)
1> (1448130730000,1448130790000,293054)
2> (1448130740000,1448130800000,294209)
Create Context and get Avro Stream from Kafka
JavaStreamingContext jssc = new JavaStreamingContext(conf,
Durations.seconds(1));
HashSet<String> topicsSet = Sets.newHashSet("FightEventTopic");
HashMap<String, String> kafkaParams = new HashMap<String, String>();
kafkaParams.put("metadata.broker.list", "xxx:11211");
kafkaParams.put("group.id", "spark");
JavaPairInputDStream<String, FightEvent> kafkaStream =
KafkaUtils.createDirectStream(jssc, String.class, FightEvent.class,
StringDecoder.class, AvroDecoder.class, kafkaParams, topicsSet);
Analyze number of hit points over a sliding window
kafkaStream.map(tuple -> tuple._2().getHitPoints())
.reduceByWindow((hit1, hit2) -> hit1 + hit2,
Durations.seconds(60), Durations.seconds(10))
.foreachRDD((rdd, time) -> {
rdd.saveAsTextFile(outputPath + "/round1-" + time.milliseconds());
LOGGER.info("Hitpoints in the last minute {}", rdd.take(5));
return null;
});
Output
20:19:32 Hitpoints in the last minute [80802]
20:19:42 Hitpoints in the last minute [101019]
20:19:52 Hitpoints in the last minute [141012]
20:20:02 Hitpoints in the last minute [184759]
20:20:12 Hitpoints in the last minute [215802]
3rd Round Points
Flink supports event time windows
Kafka and Avro worked seamlessly in both
Spark uses micro-batches, no real stream
Both have at-least-once delivery guarantees
Exactly-once depends a lot on sink/source
Round 4
Connecting Static Data with Real
Time Data
Total Hitpoints over Last Minute Per Gender
Read static data using object File and map genders
JavaRDD<Hero> staticRdd = jssc.sparkContext().objectFile(lookupPath);
JavaPairRDD<String, String> genderLookup = staticRdd.mapToPair(user -> {
int genderIndicator = user.getGender();
String gender;
switch (genderIndicator) {
case 1: gender = "MALE"; break;
case 2: gender = "FEMALE"; break;
default: gender = "OTHER"; break;
}
return new Tuple2<>(user.getId(), gender);
});
Analyze number of hit points per hitter over a sliding window
JavaPairDStream<String, Long> hitpointWindowedStream = kafkaStream
.mapToPair(tuple -> {
FightEvent fight = tuple._2();
return new Tuple2<>(fight.getHitterId(), fight.getHitPoints());
})
.reduceByKeyAndWindow((hit1, hit2) -> hit1 + hit2,
Durations.seconds(60),
Durations.seconds(10));
Join with static data to find gender for each hitter
hitpointWindowedStream.foreachRDD((rdd, time) -> {
JavaPairRDD<String, Long> hpg = rdd.leftOuterJoin(genderLookup)
.mapToPair(joinedTuple -> {
Optional<String> maybeGender = joinedTuple._2()._2();
Long hitpoints = joinedTuple._2()._1();
return new Tuple2<>(maybeGender.or("UNKNOWN"), hitpoints);
})
.reduceByKey((hit1, hit2) -> hit1 + hit2);
hpg.saveAsTextFile(outputPath + "/round2-" + time.milliseconds());
LOGGER.info("Hitpoints per gender {}", hpg.take(5));
return null;
});
Output
20:30:44 Hitpoints [(FEMALE,35869), (OTHER,435), (MALE,66226)]
20:30:54 Hitpoints [(FEMALE,48805), (OTHER,644), (MALE,87014)]
20:31:04 Hitpoints [(FEMALE,55332), (OTHER,813), (MALE,99722)]
20:31:14 Hitpoints [(FEMALE,65543), (OTHER,813), (MALE,116416)]
20:31:24 Hitpoints [(FEMALE,67507), (OTHER,813), (MALE,123750)]
Loading Static Data in Every Map
public FightEventEnricher(String bucket, String keyPrefix) {
this.bucket = bucket;
this.keyPrefix = keyPrefix;
}
@Override
public void open(Configuration parameters) {
populateHeroMapFromS3(bucket, keyPrefix);
}
@Override
public EnrichedFightEvent map(FightEvent event) throws Exception {
return new EnrichedFightEvent(event,
idToHero.get(event.getHitterId()),
idToHero.get(event.getHitteeId()));
}
private void populateHeroMapFromS3(String bucket, String keyPrefix) {
// Omitted
}
Processing Logic
hitStream.assignTimestamps(new FightEventTimestampExtractor(6000))
.map(new FightEventEnricher("s3_bucket", "output/heros"))
.filter(value -> value.getHittingHero() != null)
.keyBy(enrichedFightEvent ->
enrichedFightEvent.getHittingHero().getGender())
.timeWindow(Time.of(60, TimeUnit.SECONDS),
Time.of(10, TimeUnit.SECONDS))
.apply(new SumWindowFunction<EnrichedFightEvent, Integer>() {
@Override
public long getSummand(EnrichedFightEvent value) {
return value.getFightEvent()
.getHitPoints();
}
})
Example Output
2> (1448191350000,1448191410000,1,28478)
3> (1448191350000,1448191410000,2,264650)
2> (1448191360000,1448191420000,1,28290)
3> (1448191360000,1448191420000,2,263521)
2> (1448191370000,1448191430000,1,29327)
3> (1448191370000,1448191430000,2,265526)
4th Round Points
Spark makes combining batch and spark easier
Windowing by key works well in both
Java API of Spark can be annoying
Round 5
More Advanced Real Time
Analysis
Best Hitter over Last Minute Per Gender
Processing Logic
hitStream.assignTimestamps(new FightEventTimestampExtractor(6000))
.map(new FightEventEnricher("s3_bucket", "output/heros"))
.filter(value -> value.getHittingHero() != null)
.keyBy(fightEvent -> fightEvent.getHittingHero().getName())
.timeWindow(Time.of(60, TimeUnit.SECONDS),
Time.of(10, TimeUnit.SECONDS))
.apply(new SumWindowFunction<EnrichedFightEvent, String>() {
@Override
public long getSummand(EnrichedFightEvent value) {
return value.getFightEvent().getHitPoints();
}
})
.assignTimestamps(new AscendingTimestampExtractor<...>() {
@Override
public long extractAscendingTimestamp(Tuple4<...<tuple, long l) {
return tuple.f0;
}
})
.timeWindowAll(Time.of(10, TimeUnit.SECONDS))
.maxBy(3)
.print();
Example Output
1> (1448200070000,1448200130000,Tengu,546)
2> (1448200080000,1448200140000,Louis XIV,621)
3> (1448200090000,1448200150000,Louis XIV,561)
4> (1448200100000,1448200160000,Louis XIV,552)
5> (1448200110000,1448200170000,Phil Dexter,620)
6> (1448200120000,1448200180000,Phil Dexter,552)
7> (1448200130000,1448200190000,Kalamity,648)
8> (1448200140000,1448200200000,Jakita Wagner,656)
1> (1448200150000,1448200210000,Jakita Wagner,703)
Read static data using object File and Map names
JavaRDD<Hero> staticRdd = jssc.sparkContext().objectFile(lookupPath);
JavaPairRDD<String, String> userNameLookup = staticRdd
.mapToPair(user -> new Tuple2<>(user.getId(), user.getName()));
Analyze number of hit points per hitter over a sliding window
JavaPairDStream<String, Long> hitters = kafkaStream
.mapToPair(kafkaTuple -> new Tuple2<>(kafkaTuple._2().getHitterId(),
kafkaTuple._2().getHitPoints()))
.reduceByKeyAndWindow((accum, current) -> accum + current,
(accum, remove) -> accum - remove,
Durations.seconds(60),
Durations.seconds(10));
Join with static data to find username for each hitter
hitters.foreachRDD((rdd, time) -> {
JavaRDD<Tuple2<String, Long>> namedHitters = rdd
.leftOuterJoin(userNameLookup)
.map(joinedTuple -> {
String username = joinedTuple._2()._2().or("No name");
Long hitpoints = joinedTuple._2()._1();
return new Tuple2<>(username, hitpoints);
})
.sortBy(Tuple2::_2, false, PARTITIONS);
namedHitters.saveAsTextFile(outputPath + "/round3-" + time);
LOGGER.info("Five highest hitters (total: {}){}",
namedHitters.count(), namedHitters.take(5));
return null;
});
Output
15/11/25 20:34:23 Five highest hitters (total: 200)
[(Nick Fury,691), (Lady Blackhawk,585), (Choocho Colon,585), (Purple Man,539),
15/11/25 20:34:33 Five highest hitters (total: 378)
[(Captain Dorja,826), (Choocho Colon,773), (Nick Fury,691), (Kari Limbo,646),
15/11/25 20:34:43 Five highest hitters (total: 378)
[(Captain Dorja,1154), (Choocho Colon,867), (Wendy Go,723), (Kari Limbo,699),
15/11/25 20:34:53 Five highest hitters (total: 558)
[(Captain Dorja,1154), (Wendy Go,931), (Choocho Colon,867), (Fyodor Dostoyevsky,
Performance
Yahoo Streaming Benchmark
5th Round Points
Spark makes some things easier
But Flink is real streaming
In Spark you often have to specify partitions
The Judges' Call
Development
Compared to Hadoop, both are awesome
Both provide unified programming model for diverse scenarios
Comfort level of abstraction varies with use-case
Spark's Java API is cumbersome compared to the Scala API
Working with both is fun
Docs are ok, but spotty
Testing
Testing distributed systems will always be hard
Functionally both can be tested nicely
Monitoring
Monitoring
Community
The Judge's Call
It depends...
Use Spark, if
You have Cloudera, Hortonworks. etc support and depend on it
You want to heavily use Graph and ML libraries
You want to use the more mature project
Use Flink, if
Real-Time processing is important for your use case
You want more complex window operations
You develop in Java only
If you want to support a German project
Benchmark References
[1] http://shelan.org/blog/2016/01/31/reproducible-experiment-to-compare-apache-spark-and-apache-
flink-batch-processing/
[2] http://eastcirclek.blogspot.de/2015/06/terasort-for-spark-and-flink-with-range.html
[3] http://eastcirclek.blogspot.de/2015/07/hash-join-on-tez-spark-and-flink.html
[4] https://yahooeng.tumblr.com/post/135321837876/benchmarking-streaming-computation-engines-at
[5] http://data-artisans.com/extending-the-yahoo-streaming-benchmark/
Thank You!
Questions?
 michael.pisula@tng.tech konstantin.knauf@tng.tech

Weitere ähnliche Inhalte

Was ist angesagt?

Indexing and Query Optimizer (Mongo Austin)
Indexing and Query Optimizer (Mongo Austin)Indexing and Query Optimizer (Mongo Austin)
Indexing and Query Optimizer (Mongo Austin)MongoDB
 
Indexing and Query Optimization
Indexing and Query OptimizationIndexing and Query Optimization
Indexing and Query OptimizationMongoDB
 
Sanjar Akhmedov - Joining Infinity – Windowless Stream Processing with Flink
Sanjar Akhmedov - Joining Infinity – Windowless Stream Processing with FlinkSanjar Akhmedov - Joining Infinity – Windowless Stream Processing with Flink
Sanjar Akhmedov - Joining Infinity – Windowless Stream Processing with FlinkFlink Forward
 
Vavr Java User Group Rheinland
Vavr Java User Group RheinlandVavr Java User Group Rheinland
Vavr Java User Group RheinlandDavid Schmitz
 
Indexing & Query Optimization
Indexing & Query OptimizationIndexing & Query Optimization
Indexing & Query OptimizationMongoDB
 
Reducing Development Time with MongoDB vs. SQL
Reducing Development Time with MongoDB vs. SQLReducing Development Time with MongoDB vs. SQL
Reducing Development Time with MongoDB vs. SQLMongoDB
 
elasticsearch - advanced features in practice
elasticsearch - advanced features in practiceelasticsearch - advanced features in practice
elasticsearch - advanced features in practiceJano Suchal
 
Morphia, Spring Data & Co.
Morphia, Spring Data & Co.Morphia, Spring Data & Co.
Morphia, Spring Data & Co.Tobias Trelle
 
Java Persistence Frameworks for MongoDB
Java Persistence Frameworks for MongoDBJava Persistence Frameworks for MongoDB
Java Persistence Frameworks for MongoDBTobias Trelle
 
SH 1 - SES 3 - 3.6-Overview-Tel-Aviv.pptx
SH 1 - SES 3 - 3.6-Overview-Tel-Aviv.pptxSH 1 - SES 3 - 3.6-Overview-Tel-Aviv.pptx
SH 1 - SES 3 - 3.6-Overview-Tel-Aviv.pptxMongoDB
 
Elasticsearch & "PeopleSearch"
Elasticsearch & "PeopleSearch"Elasticsearch & "PeopleSearch"
Elasticsearch & "PeopleSearch"George Stathis
 
Spring Data JPA from 0-100 in 60 minutes
Spring Data JPA from 0-100 in 60 minutesSpring Data JPA from 0-100 in 60 minutes
Spring Data JPA from 0-100 in 60 minutesVMware Tanzu
 
Data access 2.0? Please welcome: Spring Data!
Data access 2.0? Please welcome: Spring Data!Data access 2.0? Please welcome: Spring Data!
Data access 2.0? Please welcome: Spring Data!Oliver Gierke
 
An introduction into Spring Data
An introduction into Spring DataAn introduction into Spring Data
An introduction into Spring DataOliver Gierke
 
Indexing with MongoDB
Indexing with MongoDBIndexing with MongoDB
Indexing with MongoDBMongoDB
 
Indexing and Query Optimizer (Aaron Staple)
Indexing and Query Optimizer (Aaron Staple)Indexing and Query Optimizer (Aaron Staple)
Indexing and Query Optimizer (Aaron Staple)MongoSF
 
Look Ma, “update DB to HTML5 using C++”, no hands! 
Look Ma, “update DB to HTML5 using C++”, no hands! Look Ma, “update DB to HTML5 using C++”, no hands! 
Look Ma, “update DB to HTML5 using C++”, no hands! aleks-f
 
DevFest Istanbul - a free guided tour of Neo4J
DevFest Istanbul - a free guided tour of Neo4JDevFest Istanbul - a free guided tour of Neo4J
DevFest Istanbul - a free guided tour of Neo4JFlorent Biville
 

Was ist angesagt? (20)

Indexing and Query Optimizer (Mongo Austin)
Indexing and Query Optimizer (Mongo Austin)Indexing and Query Optimizer (Mongo Austin)
Indexing and Query Optimizer (Mongo Austin)
 
Indexing and Query Optimization
Indexing and Query OptimizationIndexing and Query Optimization
Indexing and Query Optimization
 
Sanjar Akhmedov - Joining Infinity – Windowless Stream Processing with Flink
Sanjar Akhmedov - Joining Infinity – Windowless Stream Processing with FlinkSanjar Akhmedov - Joining Infinity – Windowless Stream Processing with Flink
Sanjar Akhmedov - Joining Infinity – Windowless Stream Processing with Flink
 
Vavr Java User Group Rheinland
Vavr Java User Group RheinlandVavr Java User Group Rheinland
Vavr Java User Group Rheinland
 
Indexing & Query Optimization
Indexing & Query OptimizationIndexing & Query Optimization
Indexing & Query Optimization
 
Reducing Development Time with MongoDB vs. SQL
Reducing Development Time with MongoDB vs. SQLReducing Development Time with MongoDB vs. SQL
Reducing Development Time with MongoDB vs. SQL
 
Full Text Search In PostgreSQL
Full Text Search In PostgreSQLFull Text Search In PostgreSQL
Full Text Search In PostgreSQL
 
elasticsearch - advanced features in practice
elasticsearch - advanced features in practiceelasticsearch - advanced features in practice
elasticsearch - advanced features in practice
 
Morphia, Spring Data & Co.
Morphia, Spring Data & Co.Morphia, Spring Data & Co.
Morphia, Spring Data & Co.
 
Java Persistence Frameworks for MongoDB
Java Persistence Frameworks for MongoDBJava Persistence Frameworks for MongoDB
Java Persistence Frameworks for MongoDB
 
SH 1 - SES 3 - 3.6-Overview-Tel-Aviv.pptx
SH 1 - SES 3 - 3.6-Overview-Tel-Aviv.pptxSH 1 - SES 3 - 3.6-Overview-Tel-Aviv.pptx
SH 1 - SES 3 - 3.6-Overview-Tel-Aviv.pptx
 
Elasticsearch & "PeopleSearch"
Elasticsearch & "PeopleSearch"Elasticsearch & "PeopleSearch"
Elasticsearch & "PeopleSearch"
 
Spring Data JPA from 0-100 in 60 minutes
Spring Data JPA from 0-100 in 60 minutesSpring Data JPA from 0-100 in 60 minutes
Spring Data JPA from 0-100 in 60 minutes
 
Redis
RedisRedis
Redis
 
Data access 2.0? Please welcome: Spring Data!
Data access 2.0? Please welcome: Spring Data!Data access 2.0? Please welcome: Spring Data!
Data access 2.0? Please welcome: Spring Data!
 
An introduction into Spring Data
An introduction into Spring DataAn introduction into Spring Data
An introduction into Spring Data
 
Indexing with MongoDB
Indexing with MongoDBIndexing with MongoDB
Indexing with MongoDB
 
Indexing and Query Optimizer (Aaron Staple)
Indexing and Query Optimizer (Aaron Staple)Indexing and Query Optimizer (Aaron Staple)
Indexing and Query Optimizer (Aaron Staple)
 
Look Ma, “update DB to HTML5 using C++”, no hands! 
Look Ma, “update DB to HTML5 using C++”, no hands! Look Ma, “update DB to HTML5 using C++”, no hands! 
Look Ma, “update DB to HTML5 using C++”, no hands! 
 
DevFest Istanbul - a free guided tour of Neo4J
DevFest Istanbul - a free guided tour of Neo4JDevFest Istanbul - a free guided tour of Neo4J
DevFest Istanbul - a free guided tour of Neo4J
 

Andere mochten auch

Distributed Computing and Caching in the Cloud: Hazelcast and Microsoft
Distributed Computing and Caching in the Cloud: Hazelcast and MicrosoftDistributed Computing and Caching in the Cloud: Hazelcast and Microsoft
Distributed Computing and Caching in the Cloud: Hazelcast and MicrosoftComsysto Reply GmbH
 
Getting started in Apache Spark and Flink (with Scala) - Part II
Getting started in Apache Spark and Flink (with Scala) - Part IIGetting started in Apache Spark and Flink (with Scala) - Part II
Getting started in Apache Spark and Flink (with Scala) - Part IIAlexander Panchenko
 
Grundlegende Konzepte von Elm, React und AngularDart 2 im Vergleich
Grundlegende Konzepte von Elm, React und AngularDart 2 im VergleichGrundlegende Konzepte von Elm, React und AngularDart 2 im Vergleich
Grundlegende Konzepte von Elm, React und AngularDart 2 im VergleichComsysto Reply GmbH
 
Apache Spark vs Apache Flink
Apache Spark vs Apache FlinkApache Spark vs Apache Flink
Apache Spark vs Apache FlinkAKASH SIHAG
 
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
 
Java 9 Modularity and Project Jigsaw
Java 9 Modularity and Project JigsawJava 9 Modularity and Project Jigsaw
Java 9 Modularity and Project JigsawComsysto Reply GmbH
 
Apache Flink at Strata San Jose 2016
Apache Flink at Strata San Jose 2016Apache Flink at Strata San Jose 2016
Apache Flink at Strata San Jose 2016Kostas Tzoumas
 
Continuous Processing with Apache Flink - Strata London 2016
Continuous Processing with Apache Flink - Strata London 2016Continuous Processing with Apache Flink - Strata London 2016
Continuous Processing with Apache Flink - Strata London 2016Stephan Ewen
 
HBaseCon 2012 | Lessons learned from OpenTSDB - Benoit Sigoure, StumbleUpon
HBaseCon 2012 | Lessons learned from OpenTSDB - Benoit Sigoure, StumbleUponHBaseCon 2012 | Lessons learned from OpenTSDB - Benoit Sigoure, StumbleUpon
HBaseCon 2012 | Lessons learned from OpenTSDB - Benoit Sigoure, StumbleUponCloudera, Inc.
 
Martin Junghans – Gradoop: Scalable Graph Analytics with Apache Flink
Martin Junghans – Gradoop: Scalable Graph Analytics with Apache FlinkMartin Junghans – Gradoop: Scalable Graph Analytics with Apache Flink
Martin Junghans – Gradoop: Scalable Graph Analytics with Apache FlinkFlink Forward
 
스사모 테크톡 - Apache Flink 둘러보기
스사모 테크톡 - Apache Flink 둘러보기스사모 테크톡 - Apache Flink 둘러보기
스사모 테크톡 - Apache Flink 둘러보기SangWoo Kim
 
Apache Spark & Hadoop : Train-the-trainer
Apache Spark & Hadoop : Train-the-trainerApache Spark & Hadoop : Train-the-trainer
Apache Spark & Hadoop : Train-the-trainerIMC Institute
 

Andere mochten auch (12)

Distributed Computing and Caching in the Cloud: Hazelcast and Microsoft
Distributed Computing and Caching in the Cloud: Hazelcast and MicrosoftDistributed Computing and Caching in the Cloud: Hazelcast and Microsoft
Distributed Computing and Caching in the Cloud: Hazelcast and Microsoft
 
Getting started in Apache Spark and Flink (with Scala) - Part II
Getting started in Apache Spark and Flink (with Scala) - Part IIGetting started in Apache Spark and Flink (with Scala) - Part II
Getting started in Apache Spark and Flink (with Scala) - Part II
 
Grundlegende Konzepte von Elm, React und AngularDart 2 im Vergleich
Grundlegende Konzepte von Elm, React und AngularDart 2 im VergleichGrundlegende Konzepte von Elm, React und AngularDart 2 im Vergleich
Grundlegende Konzepte von Elm, React und AngularDart 2 im Vergleich
 
Apache Spark vs Apache Flink
Apache Spark vs Apache FlinkApache Spark vs Apache Flink
Apache Spark vs Apache Flink
 
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
 
Java 9 Modularity and Project Jigsaw
Java 9 Modularity and Project JigsawJava 9 Modularity and Project Jigsaw
Java 9 Modularity and Project Jigsaw
 
Apache Flink at Strata San Jose 2016
Apache Flink at Strata San Jose 2016Apache Flink at Strata San Jose 2016
Apache Flink at Strata San Jose 2016
 
Continuous Processing with Apache Flink - Strata London 2016
Continuous Processing with Apache Flink - Strata London 2016Continuous Processing with Apache Flink - Strata London 2016
Continuous Processing with Apache Flink - Strata London 2016
 
HBaseCon 2012 | Lessons learned from OpenTSDB - Benoit Sigoure, StumbleUpon
HBaseCon 2012 | Lessons learned from OpenTSDB - Benoit Sigoure, StumbleUponHBaseCon 2012 | Lessons learned from OpenTSDB - Benoit Sigoure, StumbleUpon
HBaseCon 2012 | Lessons learned from OpenTSDB - Benoit Sigoure, StumbleUpon
 
Martin Junghans – Gradoop: Scalable Graph Analytics with Apache Flink
Martin Junghans – Gradoop: Scalable Graph Analytics with Apache FlinkMartin Junghans – Gradoop: Scalable Graph Analytics with Apache Flink
Martin Junghans – Gradoop: Scalable Graph Analytics with Apache Flink
 
스사모 테크톡 - Apache Flink 둘러보기
스사모 테크톡 - Apache Flink 둘러보기스사모 테크톡 - Apache Flink 둘러보기
스사모 테크톡 - Apache Flink 둘러보기
 
Apache Spark & Hadoop : Train-the-trainer
Apache Spark & Hadoop : Train-the-trainerApache Spark & Hadoop : Train-the-trainer
Apache Spark & Hadoop : Train-the-trainer
 

Ähnlich wie 21.04.2016 Meetup: Spark vs. Flink

Leap Ahead with Redis 6.2
Leap Ahead with Redis 6.2Leap Ahead with Redis 6.2
Leap Ahead with Redis 6.2VMware Tanzu
 
Dynamic data race detection in concurrent Java programs
Dynamic data race detection in concurrent Java programsDynamic data race detection in concurrent Java programs
Dynamic data race detection in concurrent Java programsDevexperts
 
Real-Time Spark: From Interactive Queries to Streaming
Real-Time Spark: From Interactive Queries to StreamingReal-Time Spark: From Interactive Queries to Streaming
Real-Time Spark: From Interactive Queries to StreamingDatabricks
 
A Tale of Two APIs: Using Spark Streaming In Production
A Tale of Two APIs: Using Spark Streaming In ProductionA Tale of Two APIs: Using Spark Streaming In Production
A Tale of Two APIs: Using Spark Streaming In ProductionLightbend
 
Drd secr final1_3
Drd secr final1_3Drd secr final1_3
Drd secr final1_3Devexperts
 
Hadoop & Hive Change the Data Warehousing Game Forever
Hadoop & Hive Change the Data Warehousing Game ForeverHadoop & Hive Change the Data Warehousing Game Forever
Hadoop & Hive Change the Data Warehousing Game ForeverDataWorks Summit
 
Http4s, Doobie and Circe: The Functional Web Stack
Http4s, Doobie and Circe: The Functional Web StackHttp4s, Doobie and Circe: The Functional Web Stack
Http4s, Doobie and Circe: The Functional Web StackGaryCoady
 
Scalding Big (Ad)ta
Scalding Big (Ad)taScalding Big (Ad)ta
Scalding Big (Ad)tab0ris_1
 
Big Data LDN 2017: Processing Fast Data With Apache Spark: the Tale of Two APIs
Big Data LDN 2017: Processing Fast Data With Apache Spark: the Tale of Two APIsBig Data LDN 2017: Processing Fast Data With Apache Spark: the Tale of Two APIs
Big Data LDN 2017: Processing Fast Data With Apache Spark: the Tale of Two APIsMatt Stubbs
 
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Databricks
 
Full stack analytics with Hadoop 2
Full stack analytics with Hadoop 2Full stack analytics with Hadoop 2
Full stack analytics with Hadoop 2Gabriele Modena
 
Spark schema for free with David Szakallas
Spark schema for free with David SzakallasSpark schema for free with David Szakallas
Spark schema for free with David SzakallasDatabricks
 
Spark Schema For Free with David Szakallas
 Spark Schema For Free with David Szakallas Spark Schema For Free with David Szakallas
Spark Schema For Free with David SzakallasDatabricks
 
power-assert, mechanism and philosophy
power-assert, mechanism and philosophypower-assert, mechanism and philosophy
power-assert, mechanism and philosophyTakuto Wada
 
Spark Streaming Programming Techniques You Should Know with Gerard Maas
Spark Streaming Programming Techniques You Should Know with Gerard MaasSpark Streaming Programming Techniques You Should Know with Gerard Maas
Spark Streaming Programming Techniques You Should Know with Gerard MaasSpark Summit
 
4Developers 2018: Pyt(h)on vs słoń: aktualny stan przetwarzania dużych danych...
4Developers 2018: Pyt(h)on vs słoń: aktualny stan przetwarzania dużych danych...4Developers 2018: Pyt(h)on vs słoń: aktualny stan przetwarzania dużych danych...
4Developers 2018: Pyt(h)on vs słoń: aktualny stan przetwarzania dużych danych...PROIDEA
 
GateIn Frameworks
GateIn FrameworksGateIn Frameworks
GateIn Frameworksjviet
 
A Divine Data Comedy
A Divine Data ComedyA Divine Data Comedy
A Divine Data ComedyMike Harris
 

Ähnlich wie 21.04.2016 Meetup: Spark vs. Flink (20)

Leap Ahead with Redis 6.2
Leap Ahead with Redis 6.2Leap Ahead with Redis 6.2
Leap Ahead with Redis 6.2
 
Dynamic data race detection in concurrent Java programs
Dynamic data race detection in concurrent Java programsDynamic data race detection in concurrent Java programs
Dynamic data race detection in concurrent Java programs
 
Real-Time Spark: From Interactive Queries to Streaming
Real-Time Spark: From Interactive Queries to StreamingReal-Time Spark: From Interactive Queries to Streaming
Real-Time Spark: From Interactive Queries to Streaming
 
A Tale of Two APIs: Using Spark Streaming In Production
A Tale of Two APIs: Using Spark Streaming In ProductionA Tale of Two APIs: Using Spark Streaming In Production
A Tale of Two APIs: Using Spark Streaming In Production
 
Drd secr final1_3
Drd secr final1_3Drd secr final1_3
Drd secr final1_3
 
Hadoop & Hive Change the Data Warehousing Game Forever
Hadoop & Hive Change the Data Warehousing Game ForeverHadoop & Hive Change the Data Warehousing Game Forever
Hadoop & Hive Change the Data Warehousing Game Forever
 
Http4s, Doobie and Circe: The Functional Web Stack
Http4s, Doobie and Circe: The Functional Web StackHttp4s, Doobie and Circe: The Functional Web Stack
Http4s, Doobie and Circe: The Functional Web Stack
 
Scalding Big (Ad)ta
Scalding Big (Ad)taScalding Big (Ad)ta
Scalding Big (Ad)ta
 
Big Data LDN 2017: Processing Fast Data With Apache Spark: the Tale of Two APIs
Big Data LDN 2017: Processing Fast Data With Apache Spark: the Tale of Two APIsBig Data LDN 2017: Processing Fast Data With Apache Spark: the Tale of Two APIs
Big Data LDN 2017: Processing Fast Data With Apache Spark: the Tale of Two APIs
 
Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...Easy, scalable, fault tolerant stream processing with structured streaming - ...
Easy, scalable, fault tolerant stream processing with structured streaming - ...
 
Coding Ajax
Coding AjaxCoding Ajax
Coding Ajax
 
Full stack analytics with Hadoop 2
Full stack analytics with Hadoop 2Full stack analytics with Hadoop 2
Full stack analytics with Hadoop 2
 
Spark schema for free with David Szakallas
Spark schema for free with David SzakallasSpark schema for free with David Szakallas
Spark schema for free with David Szakallas
 
Spark Schema For Free with David Szakallas
 Spark Schema For Free with David Szakallas Spark Schema For Free with David Szakallas
Spark Schema For Free with David Szakallas
 
Coding Ajax
Coding AjaxCoding Ajax
Coding Ajax
 
power-assert, mechanism and philosophy
power-assert, mechanism and philosophypower-assert, mechanism and philosophy
power-assert, mechanism and philosophy
 
Spark Streaming Programming Techniques You Should Know with Gerard Maas
Spark Streaming Programming Techniques You Should Know with Gerard MaasSpark Streaming Programming Techniques You Should Know with Gerard Maas
Spark Streaming Programming Techniques You Should Know with Gerard Maas
 
4Developers 2018: Pyt(h)on vs słoń: aktualny stan przetwarzania dużych danych...
4Developers 2018: Pyt(h)on vs słoń: aktualny stan przetwarzania dużych danych...4Developers 2018: Pyt(h)on vs słoń: aktualny stan przetwarzania dużych danych...
4Developers 2018: Pyt(h)on vs słoń: aktualny stan przetwarzania dużych danych...
 
GateIn Frameworks
GateIn FrameworksGateIn Frameworks
GateIn Frameworks
 
A Divine Data Comedy
A Divine Data ComedyA Divine Data Comedy
A Divine Data Comedy
 

Mehr von Comsysto Reply GmbH

Architectural Decisions: Smoothly and Consistently
Architectural Decisions: Smoothly and ConsistentlyArchitectural Decisions: Smoothly and Consistently
Architectural Decisions: Smoothly and ConsistentlyComsysto Reply GmbH
 
ljug-meetup-2023-03-hexagonal-architecture.pdf
ljug-meetup-2023-03-hexagonal-architecture.pdfljug-meetup-2023-03-hexagonal-architecture.pdf
ljug-meetup-2023-03-hexagonal-architecture.pdfComsysto Reply GmbH
 
Software Architecture and Architectors: useless VS valuable
Software Architecture and Architectors: useless VS valuableSoftware Architecture and Architectors: useless VS valuable
Software Architecture and Architectors: useless VS valuableComsysto Reply GmbH
 
Invited-Talk_PredAnalytics_München (2).pdf
Invited-Talk_PredAnalytics_München (2).pdfInvited-Talk_PredAnalytics_München (2).pdf
Invited-Talk_PredAnalytics_München (2).pdfComsysto Reply GmbH
 
MicroFrontends für Microservices
MicroFrontends für MicroservicesMicroFrontends für Microservices
MicroFrontends für MicroservicesComsysto Reply GmbH
 
Bable on Smart City Munich Meetup: How cities are leveraging innovative partn...
Bable on Smart City Munich Meetup: How cities are leveraging innovative partn...Bable on Smart City Munich Meetup: How cities are leveraging innovative partn...
Bable on Smart City Munich Meetup: How cities are leveraging innovative partn...Comsysto Reply GmbH
 
Smart City Munich Kickoff Meetup
Smart City Munich Kickoff Meetup Smart City Munich Kickoff Meetup
Smart City Munich Kickoff Meetup Comsysto Reply GmbH
 
Data Reliability Challenges with Spark by Henning Kropp (Spark & Hadoop User ...
Data Reliability Challenges with Spark by Henning Kropp (Spark & Hadoop User ...Data Reliability Challenges with Spark by Henning Kropp (Spark & Hadoop User ...
Data Reliability Challenges with Spark by Henning Kropp (Spark & Hadoop User ...Comsysto Reply GmbH
 
"Hadoop Data Lake vs classical Data Warehouse: How to utilize best of both wo...
"Hadoop Data Lake vs classical Data Warehouse: How to utilize best of both wo..."Hadoop Data Lake vs classical Data Warehouse: How to utilize best of both wo...
"Hadoop Data Lake vs classical Data Warehouse: How to utilize best of both wo...Comsysto Reply GmbH
 
Data lake vs Data Warehouse: Hybrid Architectures
Data lake vs Data Warehouse: Hybrid ArchitecturesData lake vs Data Warehouse: Hybrid Architectures
Data lake vs Data Warehouse: Hybrid ArchitecturesComsysto Reply GmbH
 
Building a fully-automated Fast Data Platform
Building a fully-automated Fast Data PlatformBuilding a fully-automated Fast Data Platform
Building a fully-automated Fast Data PlatformComsysto Reply GmbH
 
Apache Apex: Stream Processing Architecture and Applications
Apache Apex: Stream Processing Architecture and Applications Apache Apex: Stream Processing Architecture and Applications
Apache Apex: Stream Processing Architecture and Applications Comsysto Reply GmbH
 
Ein Prozess lernt laufen: LEGO Mindstorms Steuerung mit BPMN
Ein Prozess lernt laufen: LEGO Mindstorms Steuerung mit BPMNEin Prozess lernt laufen: LEGO Mindstorms Steuerung mit BPMN
Ein Prozess lernt laufen: LEGO Mindstorms Steuerung mit BPMNComsysto Reply GmbH
 
Geospatial applications created using java script(and nosql)
Geospatial applications created using java script(and nosql)Geospatial applications created using java script(and nosql)
Geospatial applications created using java script(and nosql)Comsysto Reply GmbH
 
Java cro 2016 - From.... to Scrum by Jurica Krizanic
Java cro 2016 - From.... to Scrum by Jurica KrizanicJava cro 2016 - From.... to Scrum by Jurica Krizanic
Java cro 2016 - From.... to Scrum by Jurica KrizanicComsysto Reply GmbH
 
Spark RDD-DF-SQL-DS-Spark Hadoop User Group Munich Meetup 2016
Spark RDD-DF-SQL-DS-Spark Hadoop User Group Munich Meetup 2016Spark RDD-DF-SQL-DS-Spark Hadoop User Group Munich Meetup 2016
Spark RDD-DF-SQL-DS-Spark Hadoop User Group Munich Meetup 2016Comsysto Reply GmbH
 
Machinelearning Spark Hadoop User Group Munich Meetup 2016
Machinelearning Spark Hadoop User Group Munich Meetup 2016Machinelearning Spark Hadoop User Group Munich Meetup 2016
Machinelearning Spark Hadoop User Group Munich Meetup 2016Comsysto Reply GmbH
 
SPARK STREAMING Spark Hadoop User Group Munich Meetup 2016
SPARK STREAMING  Spark Hadoop User Group Munich Meetup 2016SPARK STREAMING  Spark Hadoop User Group Munich Meetup 2016
SPARK STREAMING Spark Hadoop User Group Munich Meetup 2016Comsysto Reply GmbH
 
Caching and JCache with Greg Luck 18.02.16
Caching and JCache with Greg Luck 18.02.16Caching and JCache with Greg Luck 18.02.16
Caching and JCache with Greg Luck 18.02.16Comsysto Reply GmbH
 

Mehr von Comsysto Reply GmbH (20)

Architectural Decisions: Smoothly and Consistently
Architectural Decisions: Smoothly and ConsistentlyArchitectural Decisions: Smoothly and Consistently
Architectural Decisions: Smoothly and Consistently
 
ljug-meetup-2023-03-hexagonal-architecture.pdf
ljug-meetup-2023-03-hexagonal-architecture.pdfljug-meetup-2023-03-hexagonal-architecture.pdf
ljug-meetup-2023-03-hexagonal-architecture.pdf
 
Software Architecture and Architectors: useless VS valuable
Software Architecture and Architectors: useless VS valuableSoftware Architecture and Architectors: useless VS valuable
Software Architecture and Architectors: useless VS valuable
 
Invited-Talk_PredAnalytics_München (2).pdf
Invited-Talk_PredAnalytics_München (2).pdfInvited-Talk_PredAnalytics_München (2).pdf
Invited-Talk_PredAnalytics_München (2).pdf
 
MicroFrontends für Microservices
MicroFrontends für MicroservicesMicroFrontends für Microservices
MicroFrontends für Microservices
 
Alles offen = gut(ai)
Alles offen = gut(ai)Alles offen = gut(ai)
Alles offen = gut(ai)
 
Bable on Smart City Munich Meetup: How cities are leveraging innovative partn...
Bable on Smart City Munich Meetup: How cities are leveraging innovative partn...Bable on Smart City Munich Meetup: How cities are leveraging innovative partn...
Bable on Smart City Munich Meetup: How cities are leveraging innovative partn...
 
Smart City Munich Kickoff Meetup
Smart City Munich Kickoff Meetup Smart City Munich Kickoff Meetup
Smart City Munich Kickoff Meetup
 
Data Reliability Challenges with Spark by Henning Kropp (Spark & Hadoop User ...
Data Reliability Challenges with Spark by Henning Kropp (Spark & Hadoop User ...Data Reliability Challenges with Spark by Henning Kropp (Spark & Hadoop User ...
Data Reliability Challenges with Spark by Henning Kropp (Spark & Hadoop User ...
 
"Hadoop Data Lake vs classical Data Warehouse: How to utilize best of both wo...
"Hadoop Data Lake vs classical Data Warehouse: How to utilize best of both wo..."Hadoop Data Lake vs classical Data Warehouse: How to utilize best of both wo...
"Hadoop Data Lake vs classical Data Warehouse: How to utilize best of both wo...
 
Data lake vs Data Warehouse: Hybrid Architectures
Data lake vs Data Warehouse: Hybrid ArchitecturesData lake vs Data Warehouse: Hybrid Architectures
Data lake vs Data Warehouse: Hybrid Architectures
 
Building a fully-automated Fast Data Platform
Building a fully-automated Fast Data PlatformBuilding a fully-automated Fast Data Platform
Building a fully-automated Fast Data Platform
 
Apache Apex: Stream Processing Architecture and Applications
Apache Apex: Stream Processing Architecture and Applications Apache Apex: Stream Processing Architecture and Applications
Apache Apex: Stream Processing Architecture and Applications
 
Ein Prozess lernt laufen: LEGO Mindstorms Steuerung mit BPMN
Ein Prozess lernt laufen: LEGO Mindstorms Steuerung mit BPMNEin Prozess lernt laufen: LEGO Mindstorms Steuerung mit BPMN
Ein Prozess lernt laufen: LEGO Mindstorms Steuerung mit BPMN
 
Geospatial applications created using java script(and nosql)
Geospatial applications created using java script(and nosql)Geospatial applications created using java script(and nosql)
Geospatial applications created using java script(and nosql)
 
Java cro 2016 - From.... to Scrum by Jurica Krizanic
Java cro 2016 - From.... to Scrum by Jurica KrizanicJava cro 2016 - From.... to Scrum by Jurica Krizanic
Java cro 2016 - From.... to Scrum by Jurica Krizanic
 
Spark RDD-DF-SQL-DS-Spark Hadoop User Group Munich Meetup 2016
Spark RDD-DF-SQL-DS-Spark Hadoop User Group Munich Meetup 2016Spark RDD-DF-SQL-DS-Spark Hadoop User Group Munich Meetup 2016
Spark RDD-DF-SQL-DS-Spark Hadoop User Group Munich Meetup 2016
 
Machinelearning Spark Hadoop User Group Munich Meetup 2016
Machinelearning Spark Hadoop User Group Munich Meetup 2016Machinelearning Spark Hadoop User Group Munich Meetup 2016
Machinelearning Spark Hadoop User Group Munich Meetup 2016
 
SPARK STREAMING Spark Hadoop User Group Munich Meetup 2016
SPARK STREAMING  Spark Hadoop User Group Munich Meetup 2016SPARK STREAMING  Spark Hadoop User Group Munich Meetup 2016
SPARK STREAMING Spark Hadoop User Group Munich Meetup 2016
 
Caching and JCache with Greg Luck 18.02.16
Caching and JCache with Greg Luck 18.02.16Caching and JCache with Greg Luck 18.02.16
Caching and JCache with Greg Luck 18.02.16
 

Kürzlich hochgeladen

Decoding Patterns: Customer Churn Prediction Data Analysis Project
Decoding Patterns: Customer Churn Prediction Data Analysis ProjectDecoding Patterns: Customer Churn Prediction Data Analysis Project
Decoding Patterns: Customer Churn Prediction Data Analysis ProjectBoston Institute of Analytics
 
Digital Marketing Plan, how digital marketing works
Digital Marketing Plan, how digital marketing worksDigital Marketing Plan, how digital marketing works
Digital Marketing Plan, how digital marketing worksdeepakthakur548787
 
Predictive Analysis for Loan Default Presentation : Data Analysis Project PPT
Predictive Analysis for Loan Default  Presentation : Data Analysis Project PPTPredictive Analysis for Loan Default  Presentation : Data Analysis Project PPT
Predictive Analysis for Loan Default Presentation : Data Analysis Project PPTBoston Institute of Analytics
 
Bank Loan Approval Analysis: A Comprehensive Data Analysis Project
Bank Loan Approval Analysis: A Comprehensive Data Analysis ProjectBank Loan Approval Analysis: A Comprehensive Data Analysis Project
Bank Loan Approval Analysis: A Comprehensive Data Analysis ProjectBoston Institute of Analytics
 
Predicting Salary Using Data Science: A Comprehensive Analysis.pdf
Predicting Salary Using Data Science: A Comprehensive Analysis.pdfPredicting Salary Using Data Science: A Comprehensive Analysis.pdf
Predicting Salary Using Data Science: A Comprehensive Analysis.pdfBoston Institute of Analytics
 
Identifying Appropriate Test Statistics Involving Population Mean
Identifying Appropriate Test Statistics Involving Population MeanIdentifying Appropriate Test Statistics Involving Population Mean
Identifying Appropriate Test Statistics Involving Population MeanMYRABACSAFRA2
 
modul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptxmodul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptxaleedritatuxx
 
Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)Cathrine Wilhelmsen
 
Principles and Practices of Data Visualization
Principles and Practices of Data VisualizationPrinciples and Practices of Data Visualization
Principles and Practices of Data VisualizationKianJazayeri1
 
Learn How Data Science Changes Our World
Learn How Data Science Changes Our WorldLearn How Data Science Changes Our World
Learn How Data Science Changes Our WorldEduminds Learning
 
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...Boston Institute of Analytics
 
Defining Constituents, Data Vizzes and Telling a Data Story
Defining Constituents, Data Vizzes and Telling a Data StoryDefining Constituents, Data Vizzes and Telling a Data Story
Defining Constituents, Data Vizzes and Telling a Data StoryJeremy Anderson
 
Semantic Shed - Squashing and Squeezing.pptx
Semantic Shed - Squashing and Squeezing.pptxSemantic Shed - Squashing and Squeezing.pptx
Semantic Shed - Squashing and Squeezing.pptxMike Bennett
 
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Boston Institute of Analytics
 
NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...
NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...
NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...Boston Institute of Analytics
 
Thiophen Mechanism khhjjjjjjjhhhhhhhhhhh
Thiophen Mechanism khhjjjjjjjhhhhhhhhhhhThiophen Mechanism khhjjjjjjjhhhhhhhhhhh
Thiophen Mechanism khhjjjjjjjhhhhhhhhhhhYasamin16
 
Student Profile Sample report on improving academic performance by uniting gr...
Student Profile Sample report on improving academic performance by uniting gr...Student Profile Sample report on improving academic performance by uniting gr...
Student Profile Sample report on improving academic performance by uniting gr...Seán Kennedy
 
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Boston Institute of Analytics
 
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...Amil Baba Dawood bangali
 
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptxThe Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptxTasha Penwell
 

Kürzlich hochgeladen (20)

Decoding Patterns: Customer Churn Prediction Data Analysis Project
Decoding Patterns: Customer Churn Prediction Data Analysis ProjectDecoding Patterns: Customer Churn Prediction Data Analysis Project
Decoding Patterns: Customer Churn Prediction Data Analysis Project
 
Digital Marketing Plan, how digital marketing works
Digital Marketing Plan, how digital marketing worksDigital Marketing Plan, how digital marketing works
Digital Marketing Plan, how digital marketing works
 
Predictive Analysis for Loan Default Presentation : Data Analysis Project PPT
Predictive Analysis for Loan Default  Presentation : Data Analysis Project PPTPredictive Analysis for Loan Default  Presentation : Data Analysis Project PPT
Predictive Analysis for Loan Default Presentation : Data Analysis Project PPT
 
Bank Loan Approval Analysis: A Comprehensive Data Analysis Project
Bank Loan Approval Analysis: A Comprehensive Data Analysis ProjectBank Loan Approval Analysis: A Comprehensive Data Analysis Project
Bank Loan Approval Analysis: A Comprehensive Data Analysis Project
 
Predicting Salary Using Data Science: A Comprehensive Analysis.pdf
Predicting Salary Using Data Science: A Comprehensive Analysis.pdfPredicting Salary Using Data Science: A Comprehensive Analysis.pdf
Predicting Salary Using Data Science: A Comprehensive Analysis.pdf
 
Identifying Appropriate Test Statistics Involving Population Mean
Identifying Appropriate Test Statistics Involving Population MeanIdentifying Appropriate Test Statistics Involving Population Mean
Identifying Appropriate Test Statistics Involving Population Mean
 
modul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptxmodul pembelajaran robotic Workshop _ by Slidesgo.pptx
modul pembelajaran robotic Workshop _ by Slidesgo.pptx
 
Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)Data Factory in Microsoft Fabric (MsBIP #82)
Data Factory in Microsoft Fabric (MsBIP #82)
 
Principles and Practices of Data Visualization
Principles and Practices of Data VisualizationPrinciples and Practices of Data Visualization
Principles and Practices of Data Visualization
 
Learn How Data Science Changes Our World
Learn How Data Science Changes Our WorldLearn How Data Science Changes Our World
Learn How Data Science Changes Our World
 
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
Data Analysis Project : Targeting the Right Customers, Presentation on Bank M...
 
Defining Constituents, Data Vizzes and Telling a Data Story
Defining Constituents, Data Vizzes and Telling a Data StoryDefining Constituents, Data Vizzes and Telling a Data Story
Defining Constituents, Data Vizzes and Telling a Data Story
 
Semantic Shed - Squashing and Squeezing.pptx
Semantic Shed - Squashing and Squeezing.pptxSemantic Shed - Squashing and Squeezing.pptx
Semantic Shed - Squashing and Squeezing.pptx
 
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
Data Analysis Project Presentation: Unveiling Your Ideal Customer, Bank Custo...
 
NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...
NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...
NLP Data Science Project Presentation:Predicting Heart Disease with NLP Data ...
 
Thiophen Mechanism khhjjjjjjjhhhhhhhhhhh
Thiophen Mechanism khhjjjjjjjhhhhhhhhhhhThiophen Mechanism khhjjjjjjjhhhhhhhhhhh
Thiophen Mechanism khhjjjjjjjhhhhhhhhhhh
 
Student Profile Sample report on improving academic performance by uniting gr...
Student Profile Sample report on improving academic performance by uniting gr...Student Profile Sample report on improving academic performance by uniting gr...
Student Profile Sample report on improving academic performance by uniting gr...
 
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
Decoding the Heart: Student Presentation on Heart Attack Prediction with Data...
 
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
NO1 Certified Black Magic Specialist Expert Amil baba in Lahore Islamabad Raw...
 
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptxThe Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
The Power of Data-Driven Storytelling_ Unveiling the Layers of Insight.pptx
 

21.04.2016 Meetup: Spark vs. Flink

  • 1. Spark vs Flink Rumble in the (Big Data) Jungle , München, 2016-04-20 Konstantin Knauf Michael Pisula
  • 3. The Big Data Ecosystem Apache Top-Level Projects over Time 2008 2010 2013 2014 2015
  • 5. Berkeley University Origin TU Berlin 2013 Apache Incubator 04/2014 02/2014 Apache Top- Level 01/2015 databricks Company data Artisans Scala, Java, Python, R Supported languages Java, Scala, Python Scala Implemented in Java Stand-Alone, Mesos, EC2, YARN Cluster Stand-Alone, Mesos, EC2, YARN Lightning-fast cluster computing Teaser Scalable Batch and Stream Data Processing
  • 6.
  • 7.
  • 9. Real-Time Analysis of a Superhero Fight Club Fight hitter: Int hittee: Int hitpoints: Int Segment id: Int name: String segment: String Detail name: String gender: Int birthYear: Int noOfAppearances: Int Fight hitter: Int hittee: Int hitpoints: Int Fight hitter: Int hittee: Int hitpoints: Int Fight hitter: Int hittee: Int hitpoints: Int Fight hitter: Int hittee: Int hitpoints: Int Fight hitter: Int hittee: Int hitpoints: Int Fight hitter: Int hittee: Int hitpoints: Int Fight hitter: Int hittee: Int hitpoints: Int Hero id: Int name: String segment: String gender: Int birthYear: Int noOfAppearances: Int {Stream {Batch
  • 10. The Setup AWS Cluster Kafka Cluster Stream ProcessingBatch Processing Heroes Combining Stream and Batch Segment Detail Data Generator Avro Avro
  • 12. Dependencies compile "org.apache.flink:flink-java:1.0.0" compile "org.apache.flink:flink-streaming-java_2.11:1.0.0" //For Local Execution from IDE compile "org.apache.flink:flink-clients_2.11:1.0.0" Skeleton //Batch (DataSetAPI) ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); //Stream (DataStream API) StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment() //Processing Logic //For Streaming env.execute()
  • 13. Dependencies compile 'org.apache.spark:spark-core_2.10:1.5.0' compile 'org.apache.spark:spark-streaming_2.10:1.5.0' Skeleton Batch SparkConf conf = new SparkConf().setAppName(appName).setMaster(master); // Batch JavaSparkContext sparkContext = new JavaSparkContext(conf); // Stream JavaStreamingContext jssc = new JavaStreamingContext(conf, Durations.seconds(1)); // Processing Logic jssc.start(); // For Streaming
  • 14. First Impressions Practically no boiler plate Easy to get started and play around Runs in the IDE Hadoop MapReduce is much harder to get into
  • 15. Round 2 Static Data Analysis Combine both static data parts
  • 16. Read the csv file and transform it JavaRDD<String> segmentFile = sparkContext.textFile("s3://..."); JavaPairRDD<Integer, SegmentTableRecord> segmentTable = segmentFile .map(line -> line.split(",")) .filter(array -> array.length == 3) .mapToPair((String[] parts) -> { int id = Integer.parseInt(parts[0]); String name = parts[1], segment = parts[2]; return new Tuple2<>(name, new SegmentTableRecord(id, name, segment)); }); Join with detail data, filter out humans and write output segmentTable.join(detailTable) .mapValues(tuple -> { SegmentTableRecord s = tuple._1(); DetailTableRecord d = tuple._2(); return new Hero(s.getId(), s.getName(), s.getSegment(), d.getGender(), d.getBirthYear(), d.getNoOfAppearances()); }) .map(tuple -> tuple._2()) .filter(hero -> hero.getSegment().equals(HUMAN_SEGMENT)) .saveAsTextFile("s3://...");
  • 17. Loading Files from S3 into POJO DataSource<SegmentTableRecord> segmentTable = env.readCsvFile("s3://...") .ignoreInvalidLines() .pojoType(SegmentTableRecord.class, "id", "name", "segment"); Join and Filter DataSet<Hero> humanHeros = segmentTable.join(detailTable) .where("name") .equalTo("name") .with((s, d) -> new Hero(s.id, s.name, s.segment, d.gender, d.birthYear, d.noOfAppearances)) .filter(hero -> hero.segment.equals("Human")); Write back to S3 humanHeros.writeAsFormattedText(outputTablePath, WriteMode.OVERWRITE, h -> h.toCsv());
  • 18. Performance Terasort1: Flink ca 66% of runtime Terasort2: Flink ca. 68% of runtime HashJoin: Flink ca. 32% of runtime (Iterative Processes: Flink ca. 50% of runtime, ca. 7% with Delta-Iterations)
  • 19. 2nd Round Points Generally similar abstraction and feature set Flink has a nicer syntax, more sugar Spark is pretty bare-metal Flink is faster
  • 20. Round 3 Simple Real Time Analysis Total Hitpoints over Last Minute
  • 21. Configuring Environment for EventTime StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime); ExecutionConfig config = env.getConfig(); config.setAutoWatermarkInterval(500); Creating Stream from Kafka Properties properties = new Properties(); properties.put("bootstrap.servers", KAFKA_BROKERS); properties.put("zookeeper.connect", ZOOKEEPER_CONNECTION); properties.put("group.id", KAFKA_GROUP_ID); DataStreamSource<FightEvent> hitStream = env.addSource(new FlinkKafkaConsumer08<>("FightEventTopic", new FightEventDeserializer(), properties));
  • 22. Processing Logic hitStream.assignTimestamps(new FightEventTimestampExtractor(6000)) .timeWindowAll(Time.of(60, TimeUnit.SECONDS), Time.of(10, TimeUnit.SECONDS)) .apply(new SumAllWindowFunction<FightEvent>() { @Override public long getSummand(FightEvent fightEvent) { return fightEvent.getHitPoints(); } }) .writeAsCsv("s3://..."); Example Output 3> (1448130670000,1448130730000,290789) 4> (1448130680000,1448130740000,289395) 5> (1448130690000,1448130750000,291768) 6> (1448130700000,1448130760000,292634) 7> (1448130710000,1448130770000,293869) 8> (1448130720000,1448130780000,293356) 1> (1448130730000,1448130790000,293054) 2> (1448130740000,1448130800000,294209)
  • 23. Create Context and get Avro Stream from Kafka JavaStreamingContext jssc = new JavaStreamingContext(conf, Durations.seconds(1)); HashSet<String> topicsSet = Sets.newHashSet("FightEventTopic"); HashMap<String, String> kafkaParams = new HashMap<String, String>(); kafkaParams.put("metadata.broker.list", "xxx:11211"); kafkaParams.put("group.id", "spark"); JavaPairInputDStream<String, FightEvent> kafkaStream = KafkaUtils.createDirectStream(jssc, String.class, FightEvent.class, StringDecoder.class, AvroDecoder.class, kafkaParams, topicsSet); Analyze number of hit points over a sliding window kafkaStream.map(tuple -> tuple._2().getHitPoints()) .reduceByWindow((hit1, hit2) -> hit1 + hit2, Durations.seconds(60), Durations.seconds(10)) .foreachRDD((rdd, time) -> { rdd.saveAsTextFile(outputPath + "/round1-" + time.milliseconds()); LOGGER.info("Hitpoints in the last minute {}", rdd.take(5)); return null; });
  • 24. Output 20:19:32 Hitpoints in the last minute [80802] 20:19:42 Hitpoints in the last minute [101019] 20:19:52 Hitpoints in the last minute [141012] 20:20:02 Hitpoints in the last minute [184759] 20:20:12 Hitpoints in the last minute [215802]
  • 25. 3rd Round Points Flink supports event time windows Kafka and Avro worked seamlessly in both Spark uses micro-batches, no real stream Both have at-least-once delivery guarantees Exactly-once depends a lot on sink/source
  • 26. Round 4 Connecting Static Data with Real Time Data Total Hitpoints over Last Minute Per Gender
  • 27. Read static data using object File and map genders JavaRDD<Hero> staticRdd = jssc.sparkContext().objectFile(lookupPath); JavaPairRDD<String, String> genderLookup = staticRdd.mapToPair(user -> { int genderIndicator = user.getGender(); String gender; switch (genderIndicator) { case 1: gender = "MALE"; break; case 2: gender = "FEMALE"; break; default: gender = "OTHER"; break; } return new Tuple2<>(user.getId(), gender); }); Analyze number of hit points per hitter over a sliding window JavaPairDStream<String, Long> hitpointWindowedStream = kafkaStream .mapToPair(tuple -> { FightEvent fight = tuple._2(); return new Tuple2<>(fight.getHitterId(), fight.getHitPoints()); }) .reduceByKeyAndWindow((hit1, hit2) -> hit1 + hit2, Durations.seconds(60), Durations.seconds(10));
  • 28. Join with static data to find gender for each hitter hitpointWindowedStream.foreachRDD((rdd, time) -> { JavaPairRDD<String, Long> hpg = rdd.leftOuterJoin(genderLookup) .mapToPair(joinedTuple -> { Optional<String> maybeGender = joinedTuple._2()._2(); Long hitpoints = joinedTuple._2()._1(); return new Tuple2<>(maybeGender.or("UNKNOWN"), hitpoints); }) .reduceByKey((hit1, hit2) -> hit1 + hit2); hpg.saveAsTextFile(outputPath + "/round2-" + time.milliseconds()); LOGGER.info("Hitpoints per gender {}", hpg.take(5)); return null; }); Output 20:30:44 Hitpoints [(FEMALE,35869), (OTHER,435), (MALE,66226)] 20:30:54 Hitpoints [(FEMALE,48805), (OTHER,644), (MALE,87014)] 20:31:04 Hitpoints [(FEMALE,55332), (OTHER,813), (MALE,99722)] 20:31:14 Hitpoints [(FEMALE,65543), (OTHER,813), (MALE,116416)] 20:31:24 Hitpoints [(FEMALE,67507), (OTHER,813), (MALE,123750)]
  • 29. Loading Static Data in Every Map public FightEventEnricher(String bucket, String keyPrefix) { this.bucket = bucket; this.keyPrefix = keyPrefix; } @Override public void open(Configuration parameters) { populateHeroMapFromS3(bucket, keyPrefix); } @Override public EnrichedFightEvent map(FightEvent event) throws Exception { return new EnrichedFightEvent(event, idToHero.get(event.getHitterId()), idToHero.get(event.getHitteeId())); } private void populateHeroMapFromS3(String bucket, String keyPrefix) { // Omitted }
  • 30. Processing Logic hitStream.assignTimestamps(new FightEventTimestampExtractor(6000)) .map(new FightEventEnricher("s3_bucket", "output/heros")) .filter(value -> value.getHittingHero() != null) .keyBy(enrichedFightEvent -> enrichedFightEvent.getHittingHero().getGender()) .timeWindow(Time.of(60, TimeUnit.SECONDS), Time.of(10, TimeUnit.SECONDS)) .apply(new SumWindowFunction<EnrichedFightEvent, Integer>() { @Override public long getSummand(EnrichedFightEvent value) { return value.getFightEvent() .getHitPoints(); } }) Example Output 2> (1448191350000,1448191410000,1,28478) 3> (1448191350000,1448191410000,2,264650) 2> (1448191360000,1448191420000,1,28290) 3> (1448191360000,1448191420000,2,263521) 2> (1448191370000,1448191430000,1,29327) 3> (1448191370000,1448191430000,2,265526)
  • 31. 4th Round Points Spark makes combining batch and spark easier Windowing by key works well in both Java API of Spark can be annoying
  • 32. Round 5 More Advanced Real Time Analysis Best Hitter over Last Minute Per Gender
  • 33. Processing Logic hitStream.assignTimestamps(new FightEventTimestampExtractor(6000)) .map(new FightEventEnricher("s3_bucket", "output/heros")) .filter(value -> value.getHittingHero() != null) .keyBy(fightEvent -> fightEvent.getHittingHero().getName()) .timeWindow(Time.of(60, TimeUnit.SECONDS), Time.of(10, TimeUnit.SECONDS)) .apply(new SumWindowFunction<EnrichedFightEvent, String>() { @Override public long getSummand(EnrichedFightEvent value) { return value.getFightEvent().getHitPoints(); } }) .assignTimestamps(new AscendingTimestampExtractor<...>() { @Override public long extractAscendingTimestamp(Tuple4<...<tuple, long l) { return tuple.f0; } }) .timeWindowAll(Time.of(10, TimeUnit.SECONDS)) .maxBy(3) .print();
  • 34. Example Output 1> (1448200070000,1448200130000,Tengu,546) 2> (1448200080000,1448200140000,Louis XIV,621) 3> (1448200090000,1448200150000,Louis XIV,561) 4> (1448200100000,1448200160000,Louis XIV,552) 5> (1448200110000,1448200170000,Phil Dexter,620) 6> (1448200120000,1448200180000,Phil Dexter,552) 7> (1448200130000,1448200190000,Kalamity,648) 8> (1448200140000,1448200200000,Jakita Wagner,656) 1> (1448200150000,1448200210000,Jakita Wagner,703)
  • 35. Read static data using object File and Map names JavaRDD<Hero> staticRdd = jssc.sparkContext().objectFile(lookupPath); JavaPairRDD<String, String> userNameLookup = staticRdd .mapToPair(user -> new Tuple2<>(user.getId(), user.getName())); Analyze number of hit points per hitter over a sliding window JavaPairDStream<String, Long> hitters = kafkaStream .mapToPair(kafkaTuple -> new Tuple2<>(kafkaTuple._2().getHitterId(), kafkaTuple._2().getHitPoints())) .reduceByKeyAndWindow((accum, current) -> accum + current, (accum, remove) -> accum - remove, Durations.seconds(60), Durations.seconds(10));
  • 36. Join with static data to find username for each hitter hitters.foreachRDD((rdd, time) -> { JavaRDD<Tuple2<String, Long>> namedHitters = rdd .leftOuterJoin(userNameLookup) .map(joinedTuple -> { String username = joinedTuple._2()._2().or("No name"); Long hitpoints = joinedTuple._2()._1(); return new Tuple2<>(username, hitpoints); }) .sortBy(Tuple2::_2, false, PARTITIONS); namedHitters.saveAsTextFile(outputPath + "/round3-" + time); LOGGER.info("Five highest hitters (total: {}){}", namedHitters.count(), namedHitters.take(5)); return null; }); Output 15/11/25 20:34:23 Five highest hitters (total: 200) [(Nick Fury,691), (Lady Blackhawk,585), (Choocho Colon,585), (Purple Man,539), 15/11/25 20:34:33 Five highest hitters (total: 378) [(Captain Dorja,826), (Choocho Colon,773), (Nick Fury,691), (Kari Limbo,646), 15/11/25 20:34:43 Five highest hitters (total: 378) [(Captain Dorja,1154), (Choocho Colon,867), (Wendy Go,723), (Kari Limbo,699),
  • 37. 15/11/25 20:34:53 Five highest hitters (total: 558) [(Captain Dorja,1154), (Wendy Go,931), (Choocho Colon,867), (Fyodor Dostoyevsky,
  • 39. 5th Round Points Spark makes some things easier But Flink is real streaming In Spark you often have to specify partitions
  • 41. Development Compared to Hadoop, both are awesome Both provide unified programming model for diverse scenarios Comfort level of abstraction varies with use-case Spark's Java API is cumbersome compared to the Scala API Working with both is fun Docs are ok, but spotty
  • 42. Testing Testing distributed systems will always be hard Functionally both can be tested nicely
  • 46. The Judge's Call It depends...
  • 47. Use Spark, if You have Cloudera, Hortonworks. etc support and depend on it You want to heavily use Graph and ML libraries You want to use the more mature project
  • 48. Use Flink, if Real-Time processing is important for your use case You want more complex window operations You develop in Java only If you want to support a German project
  • 49. Benchmark References [1] http://shelan.org/blog/2016/01/31/reproducible-experiment-to-compare-apache-spark-and-apache- flink-batch-processing/ [2] http://eastcirclek.blogspot.de/2015/06/terasort-for-spark-and-flink-with-range.html [3] http://eastcirclek.blogspot.de/2015/07/hash-join-on-tez-spark-and-flink.html [4] https://yahooeng.tumblr.com/post/135321837876/benchmarking-streaming-computation-engines-at [5] http://data-artisans.com/extending-the-yahoo-streaming-benchmark/