SlideShare a Scribd company logo
1 of 23
Download to read offline
Introduction to Apache Beam
JB Onofré - Talend
Who am I ?
● Talend
○ Software Architect
○ Apache team
● Apache
○ Member of the Apache Software Foundation
○ Champion/Mentor/PPMC/PMC/Committer for ~ 20 projects (Beam, Falcon, Lens, Brooklyn,
Slider, Karaf, Camel, ActiveMQ, ACE, Archiva, Aries, ServiceMix, Syncope, jClouds, Unomi,
Guacamole, BatchEE, Sirona, Incubator, …)
What is Apache Beam?
1. Agnostic (unified batch + stream) Beam programming model
2. Dataflow Java SDK (soon Python, DSLs)
3. Runners for Dataflow
a. Apache Flink (thanks to data Artisans)
b. Apache Spark (thanks to Cloudera)
c. Google Cloud Dataflow (fast, no-ops)
d. Local (in-process) runner for testing
e. OSGi/Karaf
Why Apache Beam?
1. Portable - You can use the same code with different runners (abstraction) and
backends on premise, in the cloud, or locally
2. Unified - Same unified model for batch and stream processing
3. Advanced features - Event windowing, triggering, watermarking, lateless, etc.
4. Extensible model and SDK - Extensible API; can define custom sources to
read and write in parallel
Beam Programming Model
Data processing pipeline
(executed via a Beam runner)
PTransform/IO PTransform PTransformInput Output
Beam Programming Model
1. Pipelines - data processing job as a directed graph of steps
2. PCollection - the data inside a pipeline
3. Transform - a step in the pipeline (taking PCollections as input, and produce
PCollections)
a. Core transforms - common transformation provided (ParDo, GroupByKey, …)
b. Composite transforms - combine multiple transforms
c. IO transforms - endpoints of a pipeline to create PCollections (consumer/root) or use
PCollections to “write” data outside of the pipeline (producer)
Beam Programming Model - PCollection
1. PCollection is immutable, does not support random access to element, belong
to a pipeline
2. Each element in PCollection has a timestamp (set by IO Source)
3. Coder to support different data types
4. Bounded (batch) or Unbounded (streaming) PCollection (depending of the IO
Source)
5. Grouping of unbounded PCollection with Windowing (thanks to the timestamp)
a. Fixed time window
b. Sliding time window
c. Session window
d. Global window (for bounded PCollection)
e. Can deal with time skew and data lag (late data) with trigger (time-based with watermark, data-
based with counting, composite)
Beam Programming Model - IO
1. IO Sources (read data as PCollections) and Sinks (write PCollections)
2. Support Bounded and/or Unbounded PCollections
3. Provided IO - File, BigQuery, BigTable, Avro, and more coming (Kafka, JMS, …)
4. Custom IO - extensible IO API to create custom sources & sinks
5. Should deal with timestamp, watermark, deduplication, parallelism (depending
of the needs)
Apache Beam SDKs
1. API for Beam Programming Model (design pipelines, transforms, …)
2. Current SDKs
a. Java - First SDK and primary focus for refactoring and improvement
b. Python - Dataflow SDK preview for batch processing, will be migrated to Apache Beam once
the Java SDK has been stabilized (and APIs/interfaces redefined)
3. Coming (possible) SDKs/languages - Scala, Go, Ruby, etc.
4. DSLs - domain specific languages on top of the SDKs (Java fluent DSL on top
of Java SDK, …)
Java SDK
public static void main(String[] args) {
// Create a pipeline parameterized by commandline flags.
Pipeline p = Pipeline.create(PipelineOptionsFactory.fromArgs(arg));
p.apply(TextIO.Read.from("/path/to...")) // Read input.
.apply(new CountWords()) // Do some processing.
.apply(TextIO.Write.to("/path/to...")); // Write output.
// Run the pipeline.
p.run();
}
Beam Programming Model
PCollection<KV<String, Integer>> scores = input
.apply(Window.into(SessionWindows.of(Duration.standardMinutes(2))
.triggering(AtWatermark()
.withEarlyFirings(
AtPeriod(Duration.standardMinutes(1)))
.withLateFirings(AtCount(1)))
.accumulatingFiredPanes())
.apply(Sum.integersPerKey());
The Apache Beam Model (by way of the Dataflow model) includes many primitives and features which
are powerful but hard to express in other models and languages.
Runners and Backends
● Runners “translate” the code to a target backend (the runner itself doesn’t
provide the backend)
● Many runners are tied to other top-level Apache projects, such as Apache Flink
and apache Spark
● Due to this, runners can be run on-premise (on your local Flink cluster) or in a
public cloud (using Google Cloud Dataproc or Amazon EMR) for example
● Apache Beam is focused on treating runners as a top-level use case (with APIs,
support, etc.) so runners can be developed with minimal friction for maximum
pipeline portability
Beam Runners
Google Cloud Dataflow Apache Flink* Apache Spark*
[*] With varying levels of fidelity.
The Apache Beam (http://beam.incubator.apache.org) site will have more details soon.
?
Other Runner*
(local, OSGi, …)
Use Cases
Apache Beam is a great choice for both batch and stream processing and can
handle bounded and unbounded datasets
Batch can focus on ETL/ELT, catch-up processing, daily aggregations, and so on
Stream can focus on handling real-time processing on a record-by-record basis
Real use cases
● Mobile gaming data processing, both batch and stream processing (https:
//github.com/GoogleCloudPlatform/DataflowJavaSDK-examples/)
● Real-time event processing from IoT devices
Use Case - Gaming
● A game store the gaming results in the CSV file:
○ Player,team,score,timestamp
● Two pipelines:
○ UserScore (batch) sum scores for each user
○ HourlyScore (batch) similar UserScore but with a Window (hour): it calculates sum scores per
team on fixed windows.
User Game - Gaming - UserScore - Pipeline
Pipeline pipeline = Pipeline.create(options);
// Read events from a text file and parse them.
pipeline.apply(TextIO.Read.from(options.getInput()))
.apply(ParDo.named("ParseGameEvent").of(new ParseEventFn()))
// Extract and sum username/score pairs from the event data.
.apply("ExtractUserScore", new ExtractAndSumScore("user"))
.apply("WriteUserScoreSums",
new WriteToBigQuery<KV<String, Integer>>(options.
getTableName(),
configureBigQueryWrite()));
// Run the batch pipeline.
pipeline.run();
User Game - Gaming - UserScore - Avro Coder
@DefaultCoder(AvroCoder.class)
static class GameActionInfo {
@Nullable String user;
@Nullable String team;
@Nullable Integer score;
@Nullable Long timestamp;
public GameActionInfo(String user, String team, Integer score, Long
timestamp) {
…
}
…}
User Game - Gaming - UserScore - Parse Event Fn
static class ParseEventFn extends DoFn<String, GameActionInfo> {
// Log and count parse errors.
private static final Logger LOG = LoggerFactory.getLogger(ParseEventFn.class);
private final Aggregator<Long, Long> numParseErrors =
createAggregator("ParseErrors", new Sum.SumLongFn());
@Override
public void processElement(ProcessContext c) {
String[] components = c.element().split(",");
try {
String user = components[0].trim();
String team = components[1].trim();
Integer score = Integer.parseInt(components[2].trim());
Long timestamp = Long.parseLong(components[3].trim());
GameActionInfo gInfo = new GameActionInfo(user, team, score, timestamp);
c.output(gInfo);
} catch (ArrayIndexOutOfBoundsException | NumberFormatException e) {
numParseErrors.addValue(1L);
LOG.info("Parse error on " + c.element() + ", " + e.getMessage());
}
}
}
User Game - Gaming - UserScore - Sum Score Tr
public static class ExtractAndSumScore
extends PTransform<PCollection<GameActionInfo>, PCollection<KV<String, Integer>>> {
private final String field;
ExtractAndSumScore(String field) {
this.field = field;
}
@Override
public PCollection<KV<String, Integer>> apply(
PCollection<GameActionInfo> gameInfo) {
return gameInfo
.apply(MapElements
.via((GameActionInfo gInfo) -> KV.of(gInfo.getKey(field), gInfo.getScore()))
.withOutputType(new TypeDescriptor<KV<String, Integer>>() {}))
.apply(Sum.<String>integersPerKey());
}
}
User Game - Gaming - HourlyScore - Pipeline
pipeline.apply(TextIO.Read.from(options.getInput()))
.apply(ParDo.named("ParseGameEvent”).of(new ParseEventFn()))
// filter with byPredicate to ignore some data
.apply("FilterStartTime", Filter.byPredicate((GameActionInfo gInfo)
-> gInfo.getTimestamp() > startMinTimestamp.getMillis()))
.apply("FilterEndTime", Filter.byPredicate((GameActionInfo gInfo)
-> gInfo.getTimestamp() < stopMinTimestamp.getMillis()))
// use fixed-time window
.apply("AddEventTimestamps", WithTimestamps.of((GameActionInfo i) -> new Instant(i.getTimestamp())))
.apply(Window.named("FixedWindowsTeam")
.<GameActionInfo>into(FixedWindows.of(Duration.standardMinutes(60)))
// extract and sum teamname/score pairs from the event data.
.apply("ExtractTeamScore", new ExtractAndSumScore("team"))
// write the result
.apply("WriteTeamScoreSums",
new WriteWindowedToBigQuery<KV<String, Integer>>(options.getTableName(),
configureWindowedTableWrite()));
pipeline.run();
Roadmap
02/01/2016
Enter Apache
Incubator
End 2016
Cloud Dataflow
should run Beam
pipelines
Early 2016
Design for use cases,
begin refactoring
Mid 2016
Slight chaos
Late 2016
Multiple runners
execute Beam
pipelines
02/25/2016
1st commit to
ASF repository
More information and get involved!
1: Read about Apache Beam
Apache Beam website - http://beam.incubator.apache.org
2: See what the Apache Beam team is doing
Apache Beam JIRA - https://issues.apache.org/jira/browse/BEAM
Apache Beam mailing lists - http://beam.incubator.apache.org/mailing_lists/
3: Contribute!
Apache Beam git repo - https://github.com/apache/incubator-beam
Q&A

More Related Content

What's hot

Google Dataflow Intro
Google Dataflow IntroGoogle Dataflow Intro
Google Dataflow IntroIvan Glushkov
 
Cosco: An Efficient Facebook-Scale Shuffle Service
Cosco: An Efficient Facebook-Scale Shuffle ServiceCosco: An Efficient Facebook-Scale Shuffle Service
Cosco: An Efficient Facebook-Scale Shuffle ServiceDatabricks
 
Thrift vs Protocol Buffers vs Avro - Biased Comparison
Thrift vs Protocol Buffers vs Avro - Biased ComparisonThrift vs Protocol Buffers vs Avro - Biased Comparison
Thrift vs Protocol Buffers vs Avro - Biased ComparisonIgor Anishchenko
 
Data Stream Processing with Apache Flink
Data Stream Processing with Apache FlinkData Stream Processing with Apache Flink
Data Stream Processing with Apache FlinkFabian Hueske
 
Real-time Stream Processing with Apache Flink
Real-time Stream Processing with Apache FlinkReal-time Stream Processing with Apache Flink
Real-time Stream Processing with Apache FlinkDataWorks Summit
 
Streaming all over the world Real life use cases with Kafka Streams
Streaming all over the world  Real life use cases with Kafka StreamsStreaming all over the world  Real life use cases with Kafka Streams
Streaming all over the world Real life use cases with Kafka Streamsconfluent
 
Fundamentals of Apache Kafka
Fundamentals of Apache KafkaFundamentals of Apache Kafka
Fundamentals of Apache KafkaChhavi Parasher
 
HBase and HDFS: Understanding FileSystem Usage in HBase
HBase and HDFS: Understanding FileSystem Usage in HBaseHBase and HDFS: Understanding FileSystem Usage in HBase
HBase and HDFS: Understanding FileSystem Usage in HBaseenissoz
 
OSA Con 2022 - Switching Jaeger Distributed Tracing to ClickHouse to Enable A...
OSA Con 2022 - Switching Jaeger Distributed Tracing to ClickHouse to Enable A...OSA Con 2022 - Switching Jaeger Distributed Tracing to ClickHouse to Enable A...
OSA Con 2022 - Switching Jaeger Distributed Tracing to ClickHouse to Enable A...Altinity Ltd
 
Streaming Data Pipelines With Apache Beam
Streaming Data Pipelines With Apache BeamStreaming Data Pipelines With Apache Beam
Streaming Data Pipelines With Apache BeamAll Things Open
 
Google Cloud Dataflow Two Worlds Become a Much Better One
Google Cloud Dataflow Two Worlds Become a Much Better OneGoogle Cloud Dataflow Two Worlds Become a Much Better One
Google Cloud Dataflow Two Worlds Become a Much Better OneDataWorks Summit
 
Top 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsTop 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsSpark Summit
 
Airflow presentation
Airflow presentationAirflow presentation
Airflow presentationIlias Okacha
 
Google Cloud and Data Pipeline Patterns
Google Cloud and Data Pipeline PatternsGoogle Cloud and Data Pipeline Patterns
Google Cloud and Data Pipeline PatternsLynn Langit
 
Reducing Microservice Complexity with Kafka and Reactive Streams
Reducing Microservice Complexity with Kafka and Reactive StreamsReducing Microservice Complexity with Kafka and Reactive Streams
Reducing Microservice Complexity with Kafka and Reactive Streamsjimriecken
 

What's hot (20)

Kafka 101
Kafka 101Kafka 101
Kafka 101
 
Google Dataflow Intro
Google Dataflow IntroGoogle Dataflow Intro
Google Dataflow Intro
 
Cosco: An Efficient Facebook-Scale Shuffle Service
Cosco: An Efficient Facebook-Scale Shuffle ServiceCosco: An Efficient Facebook-Scale Shuffle Service
Cosco: An Efficient Facebook-Scale Shuffle Service
 
Thrift vs Protocol Buffers vs Avro - Biased Comparison
Thrift vs Protocol Buffers vs Avro - Biased ComparisonThrift vs Protocol Buffers vs Avro - Biased Comparison
Thrift vs Protocol Buffers vs Avro - Biased Comparison
 
Data Stream Processing with Apache Flink
Data Stream Processing with Apache FlinkData Stream Processing with Apache Flink
Data Stream Processing with Apache Flink
 
Apache Kafka
Apache KafkaApache Kafka
Apache Kafka
 
Real-time Stream Processing with Apache Flink
Real-time Stream Processing with Apache FlinkReal-time Stream Processing with Apache Flink
Real-time Stream Processing with Apache Flink
 
Streaming all over the world Real life use cases with Kafka Streams
Streaming all over the world  Real life use cases with Kafka StreamsStreaming all over the world  Real life use cases with Kafka Streams
Streaming all over the world Real life use cases with Kafka Streams
 
Apache Airflow
Apache AirflowApache Airflow
Apache Airflow
 
Fundamentals of Apache Kafka
Fundamentals of Apache KafkaFundamentals of Apache Kafka
Fundamentals of Apache Kafka
 
HBase and HDFS: Understanding FileSystem Usage in HBase
HBase and HDFS: Understanding FileSystem Usage in HBaseHBase and HDFS: Understanding FileSystem Usage in HBase
HBase and HDFS: Understanding FileSystem Usage in HBase
 
OSA Con 2022 - Switching Jaeger Distributed Tracing to ClickHouse to Enable A...
OSA Con 2022 - Switching Jaeger Distributed Tracing to ClickHouse to Enable A...OSA Con 2022 - Switching Jaeger Distributed Tracing to ClickHouse to Enable A...
OSA Con 2022 - Switching Jaeger Distributed Tracing to ClickHouse to Enable A...
 
Streaming Data Pipelines With Apache Beam
Streaming Data Pipelines With Apache BeamStreaming Data Pipelines With Apache Beam
Streaming Data Pipelines With Apache Beam
 
Google Cloud Dataflow Two Worlds Become a Much Better One
Google Cloud Dataflow Two Worlds Become a Much Better OneGoogle Cloud Dataflow Two Worlds Become a Much Better One
Google Cloud Dataflow Two Worlds Become a Much Better One
 
Flink Streaming
Flink StreamingFlink Streaming
Flink Streaming
 
Kafka 101
Kafka 101Kafka 101
Kafka 101
 
Top 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark ApplicationsTop 5 Mistakes When Writing Spark Applications
Top 5 Mistakes When Writing Spark Applications
 
Airflow presentation
Airflow presentationAirflow presentation
Airflow presentation
 
Google Cloud and Data Pipeline Patterns
Google Cloud and Data Pipeline PatternsGoogle Cloud and Data Pipeline Patterns
Google Cloud and Data Pipeline Patterns
 
Reducing Microservice Complexity with Kafka and Reactive Streams
Reducing Microservice Complexity with Kafka and Reactive StreamsReducing Microservice Complexity with Kafka and Reactive Streams
Reducing Microservice Complexity with Kafka and Reactive Streams
 

Similar to Introduction to Apache Beam

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
 
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
 
Realizing the Promise of Portable Data Processing with Apache Beam
Realizing the Promise of Portable Data Processing with Apache BeamRealizing the Promise of Portable Data Processing with Apache Beam
Realizing the Promise of Portable Data Processing with Apache BeamDataWorks Summit
 
HBaseCon2017 Efficient and portable data processing with Apache Beam and HBase
HBaseCon2017 Efficient and portable data processing with Apache Beam and HBaseHBaseCon2017 Efficient and portable data processing with Apache Beam and HBase
HBaseCon2017 Efficient and portable data processing with Apache Beam and HBaseHBaseCon
 
Hands on with CoAP and Californium
Hands on with CoAP and CaliforniumHands on with CoAP and Californium
Hands on with CoAP and CaliforniumJulien Vermillard
 
ApacheCon2022_Deep Dive into Building Streaming Applications with Apache Pulsar
ApacheCon2022_Deep Dive into Building Streaming Applications with Apache PulsarApacheCon2022_Deep Dive into Building Streaming Applications with Apache Pulsar
ApacheCon2022_Deep Dive into Building Streaming Applications with Apache PulsarTimothy Spann
 
Flink Forward Berlin 2018: Robert Bradshaw & Maximilian Michels - "Universal ...
Flink Forward Berlin 2018: Robert Bradshaw & Maximilian Michels - "Universal ...Flink Forward Berlin 2018: Robert Bradshaw & Maximilian Michels - "Universal ...
Flink Forward Berlin 2018: Robert Bradshaw & Maximilian Michels - "Universal ...Flink Forward
 
Python Streaming Pipelines with Beam on Flink
Python Streaming Pipelines with Beam on FlinkPython Streaming Pipelines with Beam on Flink
Python Streaming Pipelines with Beam on FlinkAljoscha Krettek
 
Logging & Metrics with Docker
Logging & Metrics with DockerLogging & Metrics with Docker
Logging & Metrics with DockerStefan Zier
 
Flink Forward Berlin 2018: Thomas Weise & Aljoscha Krettek - "Python Streamin...
Flink Forward Berlin 2018: Thomas Weise & Aljoscha Krettek - "Python Streamin...Flink Forward Berlin 2018: Thomas Weise & Aljoscha Krettek - "Python Streamin...
Flink Forward Berlin 2018: Thomas Weise & Aljoscha Krettek - "Python Streamin...Flink Forward
 
Technology Stack Discussion
Technology Stack DiscussionTechnology Stack Discussion
Technology Stack DiscussionZaiyang Li
 
Web Development Environments: Choose the best or go with the rest
Web Development Environments:  Choose the best or go with the restWeb Development Environments:  Choose the best or go with the rest
Web Development Environments: Choose the best or go with the restgeorge.james
 
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
 
Building Scalable Data Pipelines - 2016 DataPalooza Seattle
Building Scalable Data Pipelines - 2016 DataPalooza SeattleBuilding Scalable Data Pipelines - 2016 DataPalooza Seattle
Building Scalable Data Pipelines - 2016 DataPalooza SeattleEvan Chan
 
OSS EU: Deep Dive into Building Streaming Applications with Apache Pulsar
OSS EU:  Deep Dive into Building Streaming Applications with Apache PulsarOSS EU:  Deep Dive into Building Streaming Applications with Apache Pulsar
OSS EU: Deep Dive into Building Streaming Applications with Apache PulsarTimothy Spann
 
The future of server side JavaScript
The future of server side JavaScriptThe future of server side JavaScript
The future of server side JavaScriptOleg Podsechin
 
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
 
nuclio Overview October 2017
nuclio Overview October 2017nuclio Overview October 2017
nuclio Overview October 2017iguazio
 
iguazio - nuclio overview to CNCF (Sep 25th 2017)
iguazio - nuclio overview to CNCF (Sep 25th 2017)iguazio - nuclio overview to CNCF (Sep 25th 2017)
iguazio - nuclio overview to CNCF (Sep 25th 2017)Eran Duchan
 
Big Data, Data Lake, Fast Data - Dataserialiation-Formats
Big Data, Data Lake, Fast Data - Dataserialiation-FormatsBig Data, Data Lake, Fast Data - Dataserialiation-Formats
Big Data, Data Lake, Fast Data - Dataserialiation-FormatsGuido Schmutz
 

Similar to Introduction to Apache Beam (20)

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
 
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...
 
Realizing the Promise of Portable Data Processing with Apache Beam
Realizing the Promise of Portable Data Processing with Apache BeamRealizing the Promise of Portable Data Processing with Apache Beam
Realizing the Promise of Portable Data Processing with Apache Beam
 
HBaseCon2017 Efficient and portable data processing with Apache Beam and HBase
HBaseCon2017 Efficient and portable data processing with Apache Beam and HBaseHBaseCon2017 Efficient and portable data processing with Apache Beam and HBase
HBaseCon2017 Efficient and portable data processing with Apache Beam and HBase
 
Hands on with CoAP and Californium
Hands on with CoAP and CaliforniumHands on with CoAP and Californium
Hands on with CoAP and Californium
 
ApacheCon2022_Deep Dive into Building Streaming Applications with Apache Pulsar
ApacheCon2022_Deep Dive into Building Streaming Applications with Apache PulsarApacheCon2022_Deep Dive into Building Streaming Applications with Apache Pulsar
ApacheCon2022_Deep Dive into Building Streaming Applications with Apache Pulsar
 
Flink Forward Berlin 2018: Robert Bradshaw & Maximilian Michels - "Universal ...
Flink Forward Berlin 2018: Robert Bradshaw & Maximilian Michels - "Universal ...Flink Forward Berlin 2018: Robert Bradshaw & Maximilian Michels - "Universal ...
Flink Forward Berlin 2018: Robert Bradshaw & Maximilian Michels - "Universal ...
 
Python Streaming Pipelines with Beam on Flink
Python Streaming Pipelines with Beam on FlinkPython Streaming Pipelines with Beam on Flink
Python Streaming Pipelines with Beam on Flink
 
Logging & Metrics with Docker
Logging & Metrics with DockerLogging & Metrics with Docker
Logging & Metrics with Docker
 
Flink Forward Berlin 2018: Thomas Weise & Aljoscha Krettek - "Python Streamin...
Flink Forward Berlin 2018: Thomas Weise & Aljoscha Krettek - "Python Streamin...Flink Forward Berlin 2018: Thomas Weise & Aljoscha Krettek - "Python Streamin...
Flink Forward Berlin 2018: Thomas Weise & Aljoscha Krettek - "Python Streamin...
 
Technology Stack Discussion
Technology Stack DiscussionTechnology Stack Discussion
Technology Stack Discussion
 
Web Development Environments: Choose the best or go with the rest
Web Development Environments:  Choose the best or go with the restWeb Development Environments:  Choose the best or go with the rest
Web Development Environments: Choose the best or go with the rest
 
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...
 
Building Scalable Data Pipelines - 2016 DataPalooza Seattle
Building Scalable Data Pipelines - 2016 DataPalooza SeattleBuilding Scalable Data Pipelines - 2016 DataPalooza Seattle
Building Scalable Data Pipelines - 2016 DataPalooza Seattle
 
OSS EU: Deep Dive into Building Streaming Applications with Apache Pulsar
OSS EU:  Deep Dive into Building Streaming Applications with Apache PulsarOSS EU:  Deep Dive into Building Streaming Applications with Apache Pulsar
OSS EU: Deep Dive into Building Streaming Applications with Apache Pulsar
 
The future of server side JavaScript
The future of server side JavaScriptThe future of server side JavaScript
The future of server side JavaScript
 
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
 
nuclio Overview October 2017
nuclio Overview October 2017nuclio Overview October 2017
nuclio Overview October 2017
 
iguazio - nuclio overview to CNCF (Sep 25th 2017)
iguazio - nuclio overview to CNCF (Sep 25th 2017)iguazio - nuclio overview to CNCF (Sep 25th 2017)
iguazio - nuclio overview to CNCF (Sep 25th 2017)
 
Big Data, Data Lake, Fast Data - Dataserialiation-Formats
Big Data, Data Lake, Fast Data - Dataserialiation-FormatsBig Data, Data Lake, Fast Data - Dataserialiation-Formats
Big Data, Data Lake, Fast Data - Dataserialiation-Formats
 

Recently uploaded

eAuditor Audits & Inspections - conduct field inspections
eAuditor Audits & Inspections - conduct field inspectionseAuditor Audits & Inspections - conduct field inspections
eAuditor Audits & Inspections - conduct field inspectionsNirav Modi
 
Introduction-to-Software-Development-Outsourcing.pptx
Introduction-to-Software-Development-Outsourcing.pptxIntroduction-to-Software-Development-Outsourcing.pptx
Introduction-to-Software-Development-Outsourcing.pptxIntelliSource Technologies
 
Webinar_050417_LeClair12345666777889.ppt
Webinar_050417_LeClair12345666777889.pptWebinar_050417_LeClair12345666777889.ppt
Webinar_050417_LeClair12345666777889.pptkinjal48
 
IA Generativa y Grafos de Neo4j: RAG time
IA Generativa y Grafos de Neo4j: RAG timeIA Generativa y Grafos de Neo4j: RAG time
IA Generativa y Grafos de Neo4j: RAG timeNeo4j
 
Sales Territory Management: A Definitive Guide to Expand Sales Coverage
Sales Territory Management: A Definitive Guide to Expand Sales CoverageSales Territory Management: A Definitive Guide to Expand Sales Coverage
Sales Territory Management: A Definitive Guide to Expand Sales CoverageDista
 
Cybersecurity Challenges with Generative AI - for Good and Bad
Cybersecurity Challenges with Generative AI - for Good and BadCybersecurity Challenges with Generative AI - for Good and Bad
Cybersecurity Challenges with Generative AI - for Good and BadIvo Andreev
 
online pdf editor software solutions.pdf
online pdf editor software solutions.pdfonline pdf editor software solutions.pdf
online pdf editor software solutions.pdfMeon Technology
 
ARM Talk @ Rejekts - Will ARM be the new Mainstream in our Data Centers_.pdf
ARM Talk @ Rejekts - Will ARM be the new Mainstream in our Data Centers_.pdfARM Talk @ Rejekts - Will ARM be the new Mainstream in our Data Centers_.pdf
ARM Talk @ Rejekts - Will ARM be the new Mainstream in our Data Centers_.pdfTobias Schneck
 
ERP For Electrical and Electronics manufecturing.pptx
ERP For Electrical and Electronics manufecturing.pptxERP For Electrical and Electronics manufecturing.pptx
ERP For Electrical and Electronics manufecturing.pptxAutus Cyber Tech
 
Transforming PMO Success with AI - Discover OnePlan Strategic Portfolio Work ...
Transforming PMO Success with AI - Discover OnePlan Strategic Portfolio Work ...Transforming PMO Success with AI - Discover OnePlan Strategic Portfolio Work ...
Transforming PMO Success with AI - Discover OnePlan Strategic Portfolio Work ...OnePlan Solutions
 
Top Software Development Trends in 2024
Top Software Development Trends in  2024Top Software Development Trends in  2024
Top Software Development Trends in 2024Mind IT Systems
 
20240319 Car Simulator Plan.pptx . Plan for a JavaScript Car Driving Simulator.
20240319 Car Simulator Plan.pptx . Plan for a JavaScript Car Driving Simulator.20240319 Car Simulator Plan.pptx . Plan for a JavaScript Car Driving Simulator.
20240319 Car Simulator Plan.pptx . Plan for a JavaScript Car Driving Simulator.Sharon Liu
 
Optimizing Business Potential: A Guide to Outsourcing Engineering Services in...
Optimizing Business Potential: A Guide to Outsourcing Engineering Services in...Optimizing Business Potential: A Guide to Outsourcing Engineering Services in...
Optimizing Business Potential: A Guide to Outsourcing Engineering Services in...Jaydeep Chhasatia
 
OpenChain Webinar: Universal CVSS Calculator
OpenChain Webinar: Universal CVSS CalculatorOpenChain Webinar: Universal CVSS Calculator
OpenChain Webinar: Universal CVSS CalculatorShane Coughlan
 
Why Choose Brain Inventory For Ecommerce Development.pdf
Why Choose Brain Inventory For Ecommerce Development.pdfWhy Choose Brain Inventory For Ecommerce Development.pdf
Why Choose Brain Inventory For Ecommerce Development.pdfBrain Inventory
 
Deep Learning for Images with PyTorch - Datacamp
Deep Learning for Images with PyTorch - DatacampDeep Learning for Images with PyTorch - Datacamp
Deep Learning for Images with PyTorch - DatacampVICTOR MAESTRE RAMIREZ
 
Watermarking in Source Code: Applications and Security Challenges
Watermarking in Source Code: Applications and Security ChallengesWatermarking in Source Code: Applications and Security Challenges
Watermarking in Source Code: Applications and Security ChallengesShyamsundar Das
 
Big Data Bellevue Meetup | Enhancing Python Data Loading in the Cloud for AI/ML
Big Data Bellevue Meetup | Enhancing Python Data Loading in the Cloud for AI/MLBig Data Bellevue Meetup | Enhancing Python Data Loading in the Cloud for AI/ML
Big Data Bellevue Meetup | Enhancing Python Data Loading in the Cloud for AI/MLAlluxio, Inc.
 
Growing Oxen: channel operators and retries
Growing Oxen: channel operators and retriesGrowing Oxen: channel operators and retries
Growing Oxen: channel operators and retriesSoftwareMill
 

Recently uploaded (20)

eAuditor Audits & Inspections - conduct field inspections
eAuditor Audits & Inspections - conduct field inspectionseAuditor Audits & Inspections - conduct field inspections
eAuditor Audits & Inspections - conduct field inspections
 
Introduction-to-Software-Development-Outsourcing.pptx
Introduction-to-Software-Development-Outsourcing.pptxIntroduction-to-Software-Development-Outsourcing.pptx
Introduction-to-Software-Development-Outsourcing.pptx
 
Webinar_050417_LeClair12345666777889.ppt
Webinar_050417_LeClair12345666777889.pptWebinar_050417_LeClair12345666777889.ppt
Webinar_050417_LeClair12345666777889.ppt
 
IA Generativa y Grafos de Neo4j: RAG time
IA Generativa y Grafos de Neo4j: RAG timeIA Generativa y Grafos de Neo4j: RAG time
IA Generativa y Grafos de Neo4j: RAG time
 
Sales Territory Management: A Definitive Guide to Expand Sales Coverage
Sales Territory Management: A Definitive Guide to Expand Sales CoverageSales Territory Management: A Definitive Guide to Expand Sales Coverage
Sales Territory Management: A Definitive Guide to Expand Sales Coverage
 
Salesforce AI Associate Certification.pptx
Salesforce AI Associate Certification.pptxSalesforce AI Associate Certification.pptx
Salesforce AI Associate Certification.pptx
 
Cybersecurity Challenges with Generative AI - for Good and Bad
Cybersecurity Challenges with Generative AI - for Good and BadCybersecurity Challenges with Generative AI - for Good and Bad
Cybersecurity Challenges with Generative AI - for Good and Bad
 
online pdf editor software solutions.pdf
online pdf editor software solutions.pdfonline pdf editor software solutions.pdf
online pdf editor software solutions.pdf
 
ARM Talk @ Rejekts - Will ARM be the new Mainstream in our Data Centers_.pdf
ARM Talk @ Rejekts - Will ARM be the new Mainstream in our Data Centers_.pdfARM Talk @ Rejekts - Will ARM be the new Mainstream in our Data Centers_.pdf
ARM Talk @ Rejekts - Will ARM be the new Mainstream in our Data Centers_.pdf
 
ERP For Electrical and Electronics manufecturing.pptx
ERP For Electrical and Electronics manufecturing.pptxERP For Electrical and Electronics manufecturing.pptx
ERP For Electrical and Electronics manufecturing.pptx
 
Transforming PMO Success with AI - Discover OnePlan Strategic Portfolio Work ...
Transforming PMO Success with AI - Discover OnePlan Strategic Portfolio Work ...Transforming PMO Success with AI - Discover OnePlan Strategic Portfolio Work ...
Transforming PMO Success with AI - Discover OnePlan Strategic Portfolio Work ...
 
Top Software Development Trends in 2024
Top Software Development Trends in  2024Top Software Development Trends in  2024
Top Software Development Trends in 2024
 
20240319 Car Simulator Plan.pptx . Plan for a JavaScript Car Driving Simulator.
20240319 Car Simulator Plan.pptx . Plan for a JavaScript Car Driving Simulator.20240319 Car Simulator Plan.pptx . Plan for a JavaScript Car Driving Simulator.
20240319 Car Simulator Plan.pptx . Plan for a JavaScript Car Driving Simulator.
 
Optimizing Business Potential: A Guide to Outsourcing Engineering Services in...
Optimizing Business Potential: A Guide to Outsourcing Engineering Services in...Optimizing Business Potential: A Guide to Outsourcing Engineering Services in...
Optimizing Business Potential: A Guide to Outsourcing Engineering Services in...
 
OpenChain Webinar: Universal CVSS Calculator
OpenChain Webinar: Universal CVSS CalculatorOpenChain Webinar: Universal CVSS Calculator
OpenChain Webinar: Universal CVSS Calculator
 
Why Choose Brain Inventory For Ecommerce Development.pdf
Why Choose Brain Inventory For Ecommerce Development.pdfWhy Choose Brain Inventory For Ecommerce Development.pdf
Why Choose Brain Inventory For Ecommerce Development.pdf
 
Deep Learning for Images with PyTorch - Datacamp
Deep Learning for Images with PyTorch - DatacampDeep Learning for Images with PyTorch - Datacamp
Deep Learning for Images with PyTorch - Datacamp
 
Watermarking in Source Code: Applications and Security Challenges
Watermarking in Source Code: Applications and Security ChallengesWatermarking in Source Code: Applications and Security Challenges
Watermarking in Source Code: Applications and Security Challenges
 
Big Data Bellevue Meetup | Enhancing Python Data Loading in the Cloud for AI/ML
Big Data Bellevue Meetup | Enhancing Python Data Loading in the Cloud for AI/MLBig Data Bellevue Meetup | Enhancing Python Data Loading in the Cloud for AI/ML
Big Data Bellevue Meetup | Enhancing Python Data Loading in the Cloud for AI/ML
 
Growing Oxen: channel operators and retries
Growing Oxen: channel operators and retriesGrowing Oxen: channel operators and retries
Growing Oxen: channel operators and retries
 

Introduction to Apache Beam

  • 1. Introduction to Apache Beam JB Onofré - Talend
  • 2. Who am I ? ● Talend ○ Software Architect ○ Apache team ● Apache ○ Member of the Apache Software Foundation ○ Champion/Mentor/PPMC/PMC/Committer for ~ 20 projects (Beam, Falcon, Lens, Brooklyn, Slider, Karaf, Camel, ActiveMQ, ACE, Archiva, Aries, ServiceMix, Syncope, jClouds, Unomi, Guacamole, BatchEE, Sirona, Incubator, …)
  • 3. What is Apache Beam? 1. Agnostic (unified batch + stream) Beam programming model 2. Dataflow Java SDK (soon Python, DSLs) 3. Runners for Dataflow a. Apache Flink (thanks to data Artisans) b. Apache Spark (thanks to Cloudera) c. Google Cloud Dataflow (fast, no-ops) d. Local (in-process) runner for testing e. OSGi/Karaf
  • 4. Why Apache Beam? 1. Portable - You can use the same code with different runners (abstraction) and backends on premise, in the cloud, or locally 2. Unified - Same unified model for batch and stream processing 3. Advanced features - Event windowing, triggering, watermarking, lateless, etc. 4. Extensible model and SDK - Extensible API; can define custom sources to read and write in parallel
  • 5. Beam Programming Model Data processing pipeline (executed via a Beam runner) PTransform/IO PTransform PTransformInput Output
  • 6. Beam Programming Model 1. Pipelines - data processing job as a directed graph of steps 2. PCollection - the data inside a pipeline 3. Transform - a step in the pipeline (taking PCollections as input, and produce PCollections) a. Core transforms - common transformation provided (ParDo, GroupByKey, …) b. Composite transforms - combine multiple transforms c. IO transforms - endpoints of a pipeline to create PCollections (consumer/root) or use PCollections to “write” data outside of the pipeline (producer)
  • 7. Beam Programming Model - PCollection 1. PCollection is immutable, does not support random access to element, belong to a pipeline 2. Each element in PCollection has a timestamp (set by IO Source) 3. Coder to support different data types 4. Bounded (batch) or Unbounded (streaming) PCollection (depending of the IO Source) 5. Grouping of unbounded PCollection with Windowing (thanks to the timestamp) a. Fixed time window b. Sliding time window c. Session window d. Global window (for bounded PCollection) e. Can deal with time skew and data lag (late data) with trigger (time-based with watermark, data- based with counting, composite)
  • 8. Beam Programming Model - IO 1. IO Sources (read data as PCollections) and Sinks (write PCollections) 2. Support Bounded and/or Unbounded PCollections 3. Provided IO - File, BigQuery, BigTable, Avro, and more coming (Kafka, JMS, …) 4. Custom IO - extensible IO API to create custom sources & sinks 5. Should deal with timestamp, watermark, deduplication, parallelism (depending of the needs)
  • 9. Apache Beam SDKs 1. API for Beam Programming Model (design pipelines, transforms, …) 2. Current SDKs a. Java - First SDK and primary focus for refactoring and improvement b. Python - Dataflow SDK preview for batch processing, will be migrated to Apache Beam once the Java SDK has been stabilized (and APIs/interfaces redefined) 3. Coming (possible) SDKs/languages - Scala, Go, Ruby, etc. 4. DSLs - domain specific languages on top of the SDKs (Java fluent DSL on top of Java SDK, …)
  • 10. Java SDK public static void main(String[] args) { // Create a pipeline parameterized by commandline flags. Pipeline p = Pipeline.create(PipelineOptionsFactory.fromArgs(arg)); p.apply(TextIO.Read.from("/path/to...")) // Read input. .apply(new CountWords()) // Do some processing. .apply(TextIO.Write.to("/path/to...")); // Write output. // Run the pipeline. p.run(); }
  • 11. Beam Programming Model PCollection<KV<String, Integer>> scores = input .apply(Window.into(SessionWindows.of(Duration.standardMinutes(2)) .triggering(AtWatermark() .withEarlyFirings( AtPeriod(Duration.standardMinutes(1))) .withLateFirings(AtCount(1))) .accumulatingFiredPanes()) .apply(Sum.integersPerKey()); The Apache Beam Model (by way of the Dataflow model) includes many primitives and features which are powerful but hard to express in other models and languages.
  • 12. Runners and Backends ● Runners “translate” the code to a target backend (the runner itself doesn’t provide the backend) ● Many runners are tied to other top-level Apache projects, such as Apache Flink and apache Spark ● Due to this, runners can be run on-premise (on your local Flink cluster) or in a public cloud (using Google Cloud Dataproc or Amazon EMR) for example ● Apache Beam is focused on treating runners as a top-level use case (with APIs, support, etc.) so runners can be developed with minimal friction for maximum pipeline portability
  • 13. Beam Runners Google Cloud Dataflow Apache Flink* Apache Spark* [*] With varying levels of fidelity. The Apache Beam (http://beam.incubator.apache.org) site will have more details soon. ? Other Runner* (local, OSGi, …)
  • 14. Use Cases Apache Beam is a great choice for both batch and stream processing and can handle bounded and unbounded datasets Batch can focus on ETL/ELT, catch-up processing, daily aggregations, and so on Stream can focus on handling real-time processing on a record-by-record basis Real use cases ● Mobile gaming data processing, both batch and stream processing (https: //github.com/GoogleCloudPlatform/DataflowJavaSDK-examples/) ● Real-time event processing from IoT devices
  • 15. Use Case - Gaming ● A game store the gaming results in the CSV file: ○ Player,team,score,timestamp ● Two pipelines: ○ UserScore (batch) sum scores for each user ○ HourlyScore (batch) similar UserScore but with a Window (hour): it calculates sum scores per team on fixed windows.
  • 16. User Game - Gaming - UserScore - Pipeline Pipeline pipeline = Pipeline.create(options); // Read events from a text file and parse them. pipeline.apply(TextIO.Read.from(options.getInput())) .apply(ParDo.named("ParseGameEvent").of(new ParseEventFn())) // Extract and sum username/score pairs from the event data. .apply("ExtractUserScore", new ExtractAndSumScore("user")) .apply("WriteUserScoreSums", new WriteToBigQuery<KV<String, Integer>>(options. getTableName(), configureBigQueryWrite())); // Run the batch pipeline. pipeline.run();
  • 17. User Game - Gaming - UserScore - Avro Coder @DefaultCoder(AvroCoder.class) static class GameActionInfo { @Nullable String user; @Nullable String team; @Nullable Integer score; @Nullable Long timestamp; public GameActionInfo(String user, String team, Integer score, Long timestamp) { … } …}
  • 18. User Game - Gaming - UserScore - Parse Event Fn static class ParseEventFn extends DoFn<String, GameActionInfo> { // Log and count parse errors. private static final Logger LOG = LoggerFactory.getLogger(ParseEventFn.class); private final Aggregator<Long, Long> numParseErrors = createAggregator("ParseErrors", new Sum.SumLongFn()); @Override public void processElement(ProcessContext c) { String[] components = c.element().split(","); try { String user = components[0].trim(); String team = components[1].trim(); Integer score = Integer.parseInt(components[2].trim()); Long timestamp = Long.parseLong(components[3].trim()); GameActionInfo gInfo = new GameActionInfo(user, team, score, timestamp); c.output(gInfo); } catch (ArrayIndexOutOfBoundsException | NumberFormatException e) { numParseErrors.addValue(1L); LOG.info("Parse error on " + c.element() + ", " + e.getMessage()); } } }
  • 19. User Game - Gaming - UserScore - Sum Score Tr public static class ExtractAndSumScore extends PTransform<PCollection<GameActionInfo>, PCollection<KV<String, Integer>>> { private final String field; ExtractAndSumScore(String field) { this.field = field; } @Override public PCollection<KV<String, Integer>> apply( PCollection<GameActionInfo> gameInfo) { return gameInfo .apply(MapElements .via((GameActionInfo gInfo) -> KV.of(gInfo.getKey(field), gInfo.getScore())) .withOutputType(new TypeDescriptor<KV<String, Integer>>() {})) .apply(Sum.<String>integersPerKey()); } }
  • 20. User Game - Gaming - HourlyScore - Pipeline pipeline.apply(TextIO.Read.from(options.getInput())) .apply(ParDo.named("ParseGameEvent”).of(new ParseEventFn())) // filter with byPredicate to ignore some data .apply("FilterStartTime", Filter.byPredicate((GameActionInfo gInfo) -> gInfo.getTimestamp() > startMinTimestamp.getMillis())) .apply("FilterEndTime", Filter.byPredicate((GameActionInfo gInfo) -> gInfo.getTimestamp() < stopMinTimestamp.getMillis())) // use fixed-time window .apply("AddEventTimestamps", WithTimestamps.of((GameActionInfo i) -> new Instant(i.getTimestamp()))) .apply(Window.named("FixedWindowsTeam") .<GameActionInfo>into(FixedWindows.of(Duration.standardMinutes(60))) // extract and sum teamname/score pairs from the event data. .apply("ExtractTeamScore", new ExtractAndSumScore("team")) // write the result .apply("WriteTeamScoreSums", new WriteWindowedToBigQuery<KV<String, Integer>>(options.getTableName(), configureWindowedTableWrite())); pipeline.run();
  • 21. Roadmap 02/01/2016 Enter Apache Incubator End 2016 Cloud Dataflow should run Beam pipelines Early 2016 Design for use cases, begin refactoring Mid 2016 Slight chaos Late 2016 Multiple runners execute Beam pipelines 02/25/2016 1st commit to ASF repository
  • 22. More information and get involved! 1: Read about Apache Beam Apache Beam website - http://beam.incubator.apache.org 2: See what the Apache Beam team is doing Apache Beam JIRA - https://issues.apache.org/jira/browse/BEAM Apache Beam mailing lists - http://beam.incubator.apache.org/mailing_lists/ 3: Contribute! Apache Beam git repo - https://github.com/apache/incubator-beam
  • 23. Q&A