SlideShare ist ein Scribd-Unternehmen logo
1 von 44
Downloaden Sie, um offline zu lesen
streamnative.io
Pulsar Connector on Flink 1.14
The Path To Unified Batch and Streaming
Yufan Sheng
streamnative.io
This document and all information and content contained herein are
protected by China copyright and other intellectual property laws, and
are the confidential information of StreamNative, Inc. No part of this
document may be reproduced, distributed or displayed in any form or
by any means, or used to make any derivative work (such as a
translation or adaptation), without prior written permission from
StreamNative, Inc.
Copyright © 2021 StreamNative, Inc. All rights reserved.
streamnative.io
Lecturer Introduction
Yufan Sheng is a software engineer at StreamNative
where he devotes in flink or other streaming platform
integration with Apache Pulsar. Before that he was a
senior software engineer at Tencent Cloud. He was the
core committer to the Barad project (Tencent Cloud
Monitor Analysis Platform) and the leading engineer for
Barad development at Tencent.
@syhily @syhily yufan@streamnative.io
streamnative.io
Introduction to StreamNative
Developed by the original creators of Apache Pulsar
and Apache Bookkeeper, StreamNative provides a
real-time streaming platform built for Kubernetes.
StreamNative Cloud offers a turnkey platform to
help you scale mission-critical applications, on
enterprise-grade security and compliance.
streamnative.io
StreamNative is building a complete streaming solution
With Pulsar and Flink, StreamNative offers both stream storage and stream compute for a
complete streaming solution.
streamnative.io
Timeline of Pulsar connector contribution
2018/4
A simple producer
& table sink
FLINK-9168
2019/11
Pulsar catalog
FLINK-15089
2021/1
Pulsar source on legacy
SourceFunction
FLINK-20727
2018/6
Pulsar producer
FLINK-9641
2020/12
Announced on Flink
Forward 2020
Pulsar-Flink 2.7.0
2021/8
Pulsar source
on FLIP-27
FLINK-20731
streamnative.io
Subhead
streamnative.io
Unified APIs for
Batch and Stream
Processing on
Flink
streamnative.io
Batch Processing is a special case of Stream
Processing
A batch is just a bounded subset of stream(bounded stream).
streamnative.io
Batch vs Stream execution
Stream Processor
● Bounded execution
● Unbounded execution
Batch Processor
● Bounded execution
● Unbounded execution
streamnative.io
Source boundary determines Stream type
That is about 60% of the truth…
Source
Job
All source bounded
=> Job is bounded
One source unbounded
=> Job is unbounded
streamnative.io
The remaining 40% of the truth
… never seen this in
Batch Processing,
though.
The (Event-time) Watermark
*from the excellent Streaming 101 by Tyler Akidau:
https://www.oreilly.com/ideas/the-world-beyond-batch-streaming101)
streamnative.io
Inefficiencies in Job Semantics
● Watermarks, lateness, late firing, early firing, completeness vs. latency
● Incremental results vs. definitive, final results
● Processing time
As a user, you don‘t want to worry about these when
executing a bounded job on the DataStream API!
streamnative.io
The remaining 40% of the truth
Data is incomplete
Latency SLAs
Completeness and
Latency is a tradeoff
ETL pipelines, standing queries,
anomaly detection, ML
evaluation
Continuous Streaming
Data is as complete as
it gets within the job
No Low Latency SLAs
ad-hoc queries, data
exploration, ML training
Batch Processing
streamnative.io
Flink legacy (before 1.9) API stack
Streaming Runtime
DataStream API DataSet API
Batch Libraries
Table API / SQL
Streaming Libraries
● DataSet API: batch processing
● DataStream API: stream processing
● Table API/SQL: relational (batch and stream)
streamnative.io
Legacy Source Interfaces
Streaming: SourceFunction
Source
Thread
push()
Batch: InputFormat
Enumerate
splits
read splits
pull()
pull()
JobManager
streamnative.io
Batch InputFormat Processing
TaskManager TaskManager TaskManager
JobManager
(3) process split
(1) request split
(2) send split
InputFormat
createInputSplits(): splits
assignInputSplit()
openSplit(split)
nextRecord(): T
closeCurrentSplit()
streamnative.io
TaskManager TaskManager TaskManager
JobManager
(3) process split
(1) request split
(2) send split
● Splits are assigned to TaskManagers by the JobManager, which runs a copy of the
InputFormat => Flink knows about splits and can be clever about scheduling, be
reactive
● Splits can be processed in arbitrary order
● Split processing pulls records from the InputFormat
● InputFormat knows nothing about watermarks, timestamps, checkpointing => bad for
streaming
streamnative.io
Stream SourceFunction Processing
SourceFunction
run(OutputContext)
close()
(1) do your thing
TaskManager TaskManager TaskManager
● Source have a run-loop that they manage completely on their own
● Sources have flexibility and can efficiently work with the source system: batch
accesses, dealing with multiple topics from one consumer, threading model, etc…
● Flink does not know what’s going on inside and can’t be clever about it
● Sources have to implement their own per-partition watermarking, idleness tracking,
what have you
streamnative.io
A New (unified) Source Interface (FLIP-27)
Source
createSplitEnumerator()
createSplitReader()
SplitEnumerator
discoverNewSplits()
nextSplit()
isDone()
snapshotState()
SplitReader
addSplit()
hasAvailable(): Future
emitNext(Context): Status
snapshotState()
● This must support both batch and streaming use cases, allow Flink to be clever, be
able to deal with event-time, watermarks, source idiosyncrasies, and enable
snapshotting.
● This should enable new features: generic idleness detection, event-time alignment.*
* FLINK-10886: Event-time alignment for sources
streamnative.io
Execution Style
TaskManager TaskManager TaskManager
JobManager
(3) process split
(1) request split
(2) send split
streamnative.io
Execution Style
* https://www.youtube.com/watch?v=h5OYmy9Yx7Y
streamnative.io
Subhead
streamnative.io
Pulsar source
connector on
FLIP-27
streamnative.io
Pulsar Subscription Modes
Different subscription modes
have different semantics:
Exclusive/Failover - guaranteed
order, single active consumer
Shared - multiple active
consumers, no order
Key_Shared - multiple active
consumers, order for given key
Subscription D
Consumer D-1
Consumer D-2
Key-Shared
<
K
1,
V
10
>
<
K
1,
V
11
>
<
K
1,
V
12
>
<
K
2
,V
2
0
>
<
K
2
,V
2
1>
<
K
2
,V
2
2
>
Subscription C
Consumer C-1
Consumer C-2
Shared
<
K
1,
V
10
>
<
K
2
,V
2
1>
<
K
1,
V
12
>
<
K
2
,V
2
0
>
<
K
1,
V
11
>
<
K
2
,V
2
2
>
Subscription A Consumer A
Exclusive
Subscription B
Consumer B-1
Consumer B-2
In case of failure
in Consumer B-1
Failover
Pulsar Partition is also a sub set of topics which has
only one partition and could be consumed directly.
Producer 1
Producer 2
Pulsar Topic
m0
m1
m2
m3
m4
streamnative.io
Pulsar Split design
PulsarPartitionSplit
TopicPartition partition
StartCursor startCursor
StopCursor stopCursor
MessageId latestConsumedId
TxnID uncommittedTransactionId
TopicPartition
String topic
int partitionId
TopicRange range
wrap into
1 to 1
Partitioned Topic
Non-Partitioned Topic
1 to n
1 to 1
For Exclusive, Failover and Shared
Failover
Consumer B-1
Consumer B-0
Subscription B
m1
m2
m3
m4
m0
In case of failure
in Consumer B-0
Consumer A-1
Consumer A-0
Subscription A
m1
m2
m3
m4
m0
Exclusive
X
Consumer C-1
Consumer C-2
Consumer C-3
Subscription C
Shared
<
k
1
,
v
0
>
<
k
1
,
v
4
>
<
k
3
,
v
2
>
<k
2,
v1
>
<
k
2
,
v
3
>
streamnative.io
Pulsar Split design
For Key_Shared
TopicPartition
String topic
int partitionId
Partitioned Topic
Non-Partitioned Topic
1 to n
1 to 1
RangeGenerator
TopicPartition
String topic
int partitionId
TopicRange range
1 to n
…
Shared and Key_Shared can share the same subscription for all
consumers, why we create the split on partition?
Consumer D-1
Consumer D-2
Consumer D-3
Subscription D
<
k
2
,
v
1
>
<
k
2
,
v
3
>
<k3
,v2
>
<
k
1
,
v
0
>
<
k
1
,
v
4
>
Key-Shared
streamnative.io
Pulsar Split design
Create the split on partition
Consumer.seek() can only
be executed on single topic
with one partition or no
partition.
streamnative.io
Pulsar Enumerator design
Pull splits
Enumerator
TopicListSubscriber
TopicPatternSubscriber
Continuous unbound (Stream) would cycle query
Bounded (Batch) would query only once
streamnative.io
Pulsar Enumerator design
Assign split on Exclusive, Failover and Key_Shared subscription
Enumerator
Reader 1
Reader 2
Reader 3
Split 3
Split 4
Split 7
Split 2
Split 9
Split 6
Split 1
Split 5
Split 8
Consumer D-1
Consumer D-2
Consumer D-3
Subscription D
<
k
2
,
v
1
>
<
k
2
,
v
3
>
<k3
,v2
>
<
k
1
,
v
0
>
<
k
1
,
v
4
>
Key-Shared
Failover
Consumer B-1
Consumer B-0
Subscription B
m1
m2
m3
m4
m0
In case of failure
in Consumer B-0
Consumer A-1
Consumer A-0
Subscription A
m1
m2
m3
m4
m0
Exclusive
X
streamnative.io
Pulsar Enumerator design
Assign split on Shared subscription
Enumerator
Reader 1
Reader 2
Reader 3
Split 1
Split 2
Split 3
Split 1
Split 3
Split 2
Split 1
Split 2
Split 3
Why don’t share splits for Key_Shared subscription.
Consumer C-1
Consumer C-2
Consumer C-3
Subscription C
Shared
<
k
1
,
v
0
>
<
k
1
,
v
4
>
<
k
3
,
v
2
>
<k
2,
v1
>
<
k
2
,
v
3
>
streamnative.io
Pulsar Reader design
Ordered message consuming (Failover, Exclusive)
Ordered Reader
Ordered Split Reader
SortedMap<Long, Map<TopicPartition, MessageId>> cursorsToCommit
ConcurrentMap<TopicPartition, MessageId> cursorsOfFinishedSplits
ScheduledExecutorService cursorScheduler
Used cumulative message acknowledge, we only memorize last consumed message id.
Acknowledge message with id when flink finished the checkpointing.
Failover
Consumer B-1
Consumer B-0
Subscription B
m1
m2
m3
m4
m0
In case of failure
in Consumer B-0
Consumer A-1
Consumer A-0
Subscription A
m1
m2
m3
m4
m0
Exclusive
X
streamnative.io
Pulsar Reader design
Unordered message consuming (Shared, Key_Shared)
Unordered Reader
Unordered Split Reader
TransactionCoordinatorClient coordinatorClient
SortedMap<Long, List<TxnID>> transactionsToCommit
List<TxnID> transactionsOfFinishedSplits
Acknowledge message in a transaction, we only hold one transaction at a time.
Commit the transaction when flink finished the checkpointing.
Consumer D-1
Consumer D-2
Consumer D-3
Subscription D
<
k
2
,
v
1
>
<
k
2
,
v
3
>
<k3
,v2
>
<
k
1
,
v
0
>
<
k
1
,
v
4
>
Key-Shared
Consumer C-1
Consumer C-2
Consumer C-3
Subscription C
Shared
<
k
1
,
v
0
>
<
k
1
,
v
4
>
<
k
3
,
v
2
>
<k
2,
v1
>
<
k
2
,
v
3
>
streamnative.io
Flink type system vs Pulsar Schema
Pulsar
● Schema: Client side, used for serialize
and deserialize message.
● SchemaInfo: Server and Client side,
used for compatible validation and
schema evolution.
● SchemaDefinition: Client side, used for
creating a Pulsar Schema.
Flink
● (De)serializationSchema: Operator side,
used for deserialize source element and
provide a TypeInformation.
● TypeInfomation: Shared among
TaskManagers, used for message
serialization and transportation. The flink’s
default type systems, would fallback to
Kyro when no types provided.
● TypeSerializer: Used for message serialize
and deserialize.
streamnative.io
Pulsar Schema on Flink
Schema.BYTES
Reader 1
DeserializationSchema<T> Downstr
eam
Schema.BYTES
Reader 1
Schema<T>
TypeInformation<T>
Downstr
eam
streamnative.io
Pulsar Schema evolution on Flink
Schema<T>
Reader 1
TypeInformation<T> Downstr
eam
Make Pulsar Schema serializable.
Auto mapping Schema to Flink TypeInformation.
streamnative.io
Checkpoint on Pulsar Source
Job Coordinator
Enumerator
Ordered Reader Ordered Split Reader
Unordered Reader Unordered Split Reader
1. Triggered a checkpoint
2. Snapshot the split assign
status.
1. Triggered a
checkpoint
2. Snapshot
the consuming
position.
3. Save position with checkpoint id.
4. Notify checkpoint complete
5. Acknowledge message id to pulsar
1.
Triggered
a
checkpoint
2. Start a new transaction.
3. Save transaction id
with checkpoint id.
4. Notify
checkpoint
com
plete
5. Com
m
it transaction
streamnative.io
Pulsar Source configuration
We have exposed all the configuration options in:
• PulsarSourceOptions: Pulsar Source and Pulsar Consumer config option.
• PulsarOptions: Options for PulsarAdmin and PulsarClient builder.
These options are define by flink’s ConfigOption, with strong type support.
streamnative.io
Overview of PulsarSourceBuilder
PulsarSource<String> source = PulsarSource
.builder()
.setServiceUrl(PULSAR_BROKER_URL)
.setAdminUrl(PULSAR_BROKER_HTTP_URL)
.setSubscriptionName("flink-source-1")
.setTopics(Arrays.asList(TOPIC1, TOPIC2))
.setDeserializationSchema(PulsarDeserializationSchema.flinkSchema(new SimpleStringSchema()))
.setUnbounded(StopCursor.atEventTime(System.currentTimeMillis()))
.build();
A simple sample on consuming string messages from Pulsar
streamnative.io
You should provide at least these
information.
setServiceUrl
setAdminUrl
setSubscriptionName
setTopics/setTopicPattern
setDeserializationSchema
Overview of PulsarSourceBuilder
streamnative.io
Create PulsarDeserializationSchema
You can create a PulsarDeserializationSchema in three ways:
1. Using a Pulsar Schema, we would auto create it’s related flink TypeInformation.
2. Using a Flink DeserializationSchema, we would use pure flink mechanism.
3. Using a Flink TypeInformation, it’s not recommend for it doesn’t compatible with others above.
streamnative.io
Provide StartCursor and StopCursor
StartCursor could be created in four ways.
1. earliest(): From the earliest available message in the topic.
2. latest(): From the latest available message in the topic.
3. fromMessageId(MessageId messageId, boolean inclusive): From a specified message id.
4. fromMessageTime(long timestamp): From a specified message time.
Caution:
1. If the provided message id/time is bigger than the available message, we would start
consuming from the latest().
2. If the provided message id/time doesn’t existed (deleted), we would find the next
available message, it may fallback to the latest() finally.
streamnative.io
Provide StartCursor and StopCursor
StopCursor could be created in five ways.
1. never(): We never stop, the source is infinite.
2. latest(): Query the latest message and stop after this message id.
3. atMessageId(MessageId messageId): Stop at a specified message id (exclude).
4. afterMessageId(MessageId messageId): Stop after a specified message id (include).
5. atEventTime(long timestamp): Stop at a specified message time (exclude).
Caution:
latest() is used for test and batch data consuming. You should fully understand its
internal logic before using.
streamnative.io
Follow us
StreamNative Apache Pulsar Pulsar Bot
streamnative.io
streamnative.io
streamnative.io
Thank You

Weitere ähnliche Inhalte

Was ist angesagt?

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
 
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords   The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords Stephan Ewen
 
Maximilian Michels – Google Cloud Dataflow on Top of Apache Flink
Maximilian Michels – Google Cloud Dataflow on Top of Apache FlinkMaximilian Michels – Google Cloud Dataflow on Top of Apache Flink
Maximilian Michels – Google Cloud Dataflow on Top of Apache FlinkFlink Forward
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internalsKostas Tzoumas
 
Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015
Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015
Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015Till Rohrmann
 
Till Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
Till Rohrmann – Fault Tolerance and Job Recovery in Apache FlinkTill Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
Till Rohrmann – Fault Tolerance and Job Recovery in Apache FlinkFlink Forward
 
Virtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy Farkas
Virtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy FarkasVirtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy Farkas
Virtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy FarkasFlink Forward
 
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache FlinkTzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache FlinkVerverica
 
Tran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
Tran Nam-Luc – Stale Synchronous Parallel Iterations on FlinkTran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
Tran Nam-Luc – Stale Synchronous Parallel Iterations on FlinkFlink Forward
 
Flink Streaming @BudapestData
Flink Streaming @BudapestDataFlink Streaming @BudapestData
Flink Streaming @BudapestDataGyula Fóra
 
Apache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World LondonApache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World LondonStephan Ewen
 
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Rei...
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck -  Pravega: Storage Rei...Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck -  Pravega: Storage Rei...
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Rei...Flink Forward
 
Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...
Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...
Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...Flink Forward
 
Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...
Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...
Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...Flink Forward
 
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015Robert Metzger
 
Matthias J. Sax – A Tale of Squirrels and Storms
Matthias J. Sax – A Tale of Squirrels and StormsMatthias J. Sax – A Tale of Squirrels and Storms
Matthias J. Sax – A Tale of Squirrels and StormsFlink Forward
 
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud" Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud" Flink Forward
 
Fault Tolerance and Job Recovery in Apache Flink @ FlinkForward 2015
Fault Tolerance and Job Recovery in Apache Flink @ FlinkForward 2015Fault Tolerance and Job Recovery in Apache Flink @ FlinkForward 2015
Fault Tolerance and Job Recovery in Apache Flink @ FlinkForward 2015Till Rohrmann
 

Was ist angesagt? (20)

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
 
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords   The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
 
Maximilian Michels – Google Cloud Dataflow on Top of Apache Flink
Maximilian Michels – Google Cloud Dataflow on Top of Apache FlinkMaximilian Michels – Google Cloud Dataflow on Top of Apache Flink
Maximilian Michels – Google Cloud Dataflow on Top of Apache Flink
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internals
 
The Stream Processor as a Database Apache Flink
The Stream Processor as a Database Apache FlinkThe Stream Processor as a Database Apache Flink
The Stream Processor as a Database Apache Flink
 
Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015
Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015
Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015
 
Till Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
Till Rohrmann – Fault Tolerance and Job Recovery in Apache FlinkTill Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
Till Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
 
Virtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy Farkas
Virtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy FarkasVirtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy Farkas
Virtual Flink Forward 2020: Autoscaling Flink at Netflix - Timothy Farkas
 
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache FlinkTzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
 
Tran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
Tran Nam-Luc – Stale Synchronous Parallel Iterations on FlinkTran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
Tran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
 
Flink Streaming @BudapestData
Flink Streaming @BudapestDataFlink Streaming @BudapestData
Flink Streaming @BudapestData
 
Apache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World LondonApache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World London
 
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Rei...
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck -  Pravega: Storage Rei...Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck -  Pravega: Storage Rei...
Flink Forward SF 2017: Srikanth Satya & Tom Kaitchuck - Pravega: Storage Rei...
 
Zurich Flink Meetup
Zurich Flink MeetupZurich Flink Meetup
Zurich Flink Meetup
 
Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...
Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...
Keynote: Building and Operating A Serverless Streaming Runtime for Apache Bea...
 
Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...
Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...
Flink Forward San Francisco 2018: Stefan Richter - "How to build a modern str...
 
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
 
Matthias J. Sax – A Tale of Squirrels and Storms
Matthias J. Sax – A Tale of Squirrels and StormsMatthias J. Sax – A Tale of Squirrels and Storms
Matthias J. Sax – A Tale of Squirrels and Storms
 
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud" Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
Flink Forward San Francisco 2018: Steven Wu - "Scaling Flink in Cloud"
 
Fault Tolerance and Job Recovery in Apache Flink @ FlinkForward 2015
Fault Tolerance and Job Recovery in Apache Flink @ FlinkForward 2015Fault Tolerance and Job Recovery in Apache Flink @ FlinkForward 2015
Fault Tolerance and Job Recovery in Apache Flink @ FlinkForward 2015
 

Ähnlich wie Pulsar connector on flink 1.14

Ai dev world utilizing apache pulsar, apache ni fi and minifi for edgeai io...
Ai dev world   utilizing apache pulsar, apache ni fi and minifi for edgeai io...Ai dev world   utilizing apache pulsar, apache ni fi and minifi for edgeai io...
Ai dev world utilizing apache pulsar, apache ni fi and minifi for edgeai io...Timothy Spann
 
Apache Flink Training Workshop @ HadoopCon2016 - #1 System Overview
Apache Flink Training Workshop @ HadoopCon2016 - #1 System OverviewApache Flink Training Workshop @ HadoopCon2016 - #1 System Overview
Apache Flink Training Workshop @ HadoopCon2016 - #1 System OverviewApache Flink Taiwan User Group
 
The Beam Vision for Portability: "Write once run anywhere"
The Beam Vision for Portability: "Write once run anywhere"The Beam Vision for Portability: "Write once run anywhere"
The Beam Vision for Portability: "Write once run anywhere"Knoldus Inc.
 
Talk Python To Me: Stream Processing in your favourite Language with Beam on ...
Talk Python To Me: Stream Processing in your favourite Language with Beam on ...Talk Python To Me: Stream Processing in your favourite Language with Beam on ...
Talk Python To Me: Stream Processing in your favourite Language with Beam on ...Aljoscha Krettek
 
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
 
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
 
Fault Tolerance at Speed
Fault Tolerance at SpeedFault Tolerance at Speed
Fault Tolerance at SpeedC4Media
 
Flink Forward Berlin 2017: Aljoscha Krettek - Talk Python to me: Stream Proce...
Flink Forward Berlin 2017: Aljoscha Krettek - Talk Python to me: Stream Proce...Flink Forward Berlin 2017: Aljoscha Krettek - Talk Python to me: Stream Proce...
Flink Forward Berlin 2017: Aljoscha Krettek - Talk Python to me: Stream Proce...Flink Forward
 
Real time-collaborative-editor-presentation
Real time-collaborative-editor-presentationReal time-collaborative-editor-presentation
Real time-collaborative-editor-presentationbflueras
 
Advanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 Keynote
Advanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 KeynoteAdvanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 Keynote
Advanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 KeynoteStreamNative
 
Project Reactor Now and Tomorrow
Project Reactor Now and TomorrowProject Reactor Now and Tomorrow
Project Reactor Now and TomorrowVMware Tanzu
 
Streaming Dataflow with Apache Flink
Streaming Dataflow with Apache Flink Streaming Dataflow with Apache Flink
Streaming Dataflow with Apache Flink huguk
 
Near real-time anomaly detection at Lyft
Near real-time anomaly detection at LyftNear real-time anomaly detection at Lyft
Near real-time anomaly detection at Lyftmarkgrover
 
Counting Elements in Streams
Counting Elements in StreamsCounting Elements in Streams
Counting Elements in StreamsJamie Grier
 
Tornado Web Server Internals
Tornado Web Server InternalsTornado Web Server Internals
Tornado Web Server InternalsPraveen Gollakota
 
BigQuery case study in Groovenauts & Dive into the DataflowJavaSDK
BigQuery case study in Groovenauts & Dive into the DataflowJavaSDKBigQuery case study in Groovenauts & Dive into the DataflowJavaSDK
BigQuery case study in Groovenauts & Dive into the DataflowJavaSDKnagachika t
 
[March sn meetup] apache pulsar + apache nifi for cloud data lake
[March sn meetup] apache pulsar + apache nifi for cloud data lake[March sn meetup] apache pulsar + apache nifi for cloud data lake
[March sn meetup] apache pulsar + apache nifi for cloud data lakeTimothy Spann
 
Apache Pulsar Development 101 with Python
Apache Pulsar Development 101 with PythonApache Pulsar Development 101 with Python
Apache Pulsar Development 101 with PythonTimothy Spann
 

Ähnlich wie Pulsar connector on flink 1.14 (20)

Ai dev world utilizing apache pulsar, apache ni fi and minifi for edgeai io...
Ai dev world   utilizing apache pulsar, apache ni fi and minifi for edgeai io...Ai dev world   utilizing apache pulsar, apache ni fi and minifi for edgeai io...
Ai dev world utilizing apache pulsar, apache ni fi and minifi for edgeai io...
 
Apache Flink Training Workshop @ HadoopCon2016 - #1 System Overview
Apache Flink Training Workshop @ HadoopCon2016 - #1 System OverviewApache Flink Training Workshop @ HadoopCon2016 - #1 System Overview
Apache Flink Training Workshop @ HadoopCon2016 - #1 System Overview
 
FLiP Into Trino
FLiP Into TrinoFLiP Into Trino
FLiP Into Trino
 
The Beam Vision for Portability: "Write once run anywhere"
The Beam Vision for Portability: "Write once run anywhere"The Beam Vision for Portability: "Write once run anywhere"
The Beam Vision for Portability: "Write once run anywhere"
 
Talk Python To Me: Stream Processing in your favourite Language with Beam on ...
Talk Python To Me: Stream Processing in your favourite Language with Beam on ...Talk Python To Me: Stream Processing in your favourite Language with Beam on ...
Talk Python To Me: Stream Processing in your favourite Language with Beam on ...
 
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...
 
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
 
Fault Tolerance at Speed
Fault Tolerance at SpeedFault Tolerance at Speed
Fault Tolerance at Speed
 
Flink Forward Berlin 2017: Aljoscha Krettek - Talk Python to me: Stream Proce...
Flink Forward Berlin 2017: Aljoscha Krettek - Talk Python to me: Stream Proce...Flink Forward Berlin 2017: Aljoscha Krettek - Talk Python to me: Stream Proce...
Flink Forward Berlin 2017: Aljoscha Krettek - Talk Python to me: Stream Proce...
 
Real time-collaborative-editor-presentation
Real time-collaborative-editor-presentationReal time-collaborative-editor-presentation
Real time-collaborative-editor-presentation
 
Advanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 Keynote
Advanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 KeynoteAdvanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 Keynote
Advanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 Keynote
 
Explore Android Internals
Explore Android InternalsExplore Android Internals
Explore Android Internals
 
Project Reactor Now and Tomorrow
Project Reactor Now and TomorrowProject Reactor Now and Tomorrow
Project Reactor Now and Tomorrow
 
Streaming Dataflow with Apache Flink
Streaming Dataflow with Apache Flink Streaming Dataflow with Apache Flink
Streaming Dataflow with Apache Flink
 
Near real-time anomaly detection at Lyft
Near real-time anomaly detection at LyftNear real-time anomaly detection at Lyft
Near real-time anomaly detection at Lyft
 
Counting Elements in Streams
Counting Elements in StreamsCounting Elements in Streams
Counting Elements in Streams
 
Tornado Web Server Internals
Tornado Web Server InternalsTornado Web Server Internals
Tornado Web Server Internals
 
BigQuery case study in Groovenauts & Dive into the DataflowJavaSDK
BigQuery case study in Groovenauts & Dive into the DataflowJavaSDKBigQuery case study in Groovenauts & Dive into the DataflowJavaSDK
BigQuery case study in Groovenauts & Dive into the DataflowJavaSDK
 
[March sn meetup] apache pulsar + apache nifi for cloud data lake
[March sn meetup] apache pulsar + apache nifi for cloud data lake[March sn meetup] apache pulsar + apache nifi for cloud data lake
[March sn meetup] apache pulsar + apache nifi for cloud data lake
 
Apache Pulsar Development 101 with Python
Apache Pulsar Development 101 with PythonApache Pulsar Development 101 with Python
Apache Pulsar Development 101 with Python
 

Kürzlich hochgeladen

Emixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native developmentEmixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native developmentPim van der Noll
 
Digital Tools & AI in Career Development
Digital Tools & AI in Career DevelopmentDigital Tools & AI in Career Development
Digital Tools & AI in Career DevelopmentMahmoud Rabie
 
A Framework for Development in the AI Age
A Framework for Development in the AI AgeA Framework for Development in the AI Age
A Framework for Development in the AI AgeCprime
 
Modern Roaming for Notes and Nomad – Cheaper Faster Better Stronger
Modern Roaming for Notes and Nomad – Cheaper Faster Better StrongerModern Roaming for Notes and Nomad – Cheaper Faster Better Stronger
Modern Roaming for Notes and Nomad – Cheaper Faster Better Strongerpanagenda
 
Genislab builds better products and faster go-to-market with Lean project man...
Genislab builds better products and faster go-to-market with Lean project man...Genislab builds better products and faster go-to-market with Lean project man...
Genislab builds better products and faster go-to-market with Lean project man...Farhan Tariq
 
Design pattern talk by Kaya Weers - 2024 (v2)
Design pattern talk by Kaya Weers - 2024 (v2)Design pattern talk by Kaya Weers - 2024 (v2)
Design pattern talk by Kaya Weers - 2024 (v2)Kaya Weers
 
Generative AI - Gitex v1Generative AI - Gitex v1.pptx
Generative AI - Gitex v1Generative AI - Gitex v1.pptxGenerative AI - Gitex v1Generative AI - Gitex v1.pptx
Generative AI - Gitex v1Generative AI - Gitex v1.pptxfnnc6jmgwh
 
Landscape Catalogue 2024 Australia-1.pdf
Landscape Catalogue 2024 Australia-1.pdfLandscape Catalogue 2024 Australia-1.pdf
Landscape Catalogue 2024 Australia-1.pdfAarwolf Industries LLC
 
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24Mark Goldstein
 
Microsoft 365 Copilot: How to boost your productivity with AI – Part one: Ado...
Microsoft 365 Copilot: How to boost your productivity with AI – Part one: Ado...Microsoft 365 Copilot: How to boost your productivity with AI – Part one: Ado...
Microsoft 365 Copilot: How to boost your productivity with AI – Part one: Ado...Nikki Chapple
 
QCon London: Mastering long-running processes in modern architectures
QCon London: Mastering long-running processes in modern architecturesQCon London: Mastering long-running processes in modern architectures
QCon London: Mastering long-running processes in modern architecturesBernd Ruecker
 
4. Cobus Valentine- Cybersecurity Threats and Solutions for the Public Sector
4. Cobus Valentine- Cybersecurity Threats and Solutions for the Public Sector4. Cobus Valentine- Cybersecurity Threats and Solutions for the Public Sector
4. Cobus Valentine- Cybersecurity Threats and Solutions for the Public Sectoritnewsafrica
 
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyesHow to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyesThousandEyes
 
Bridging Between CAD & GIS: 6 Ways to Automate Your Data Integration
Bridging Between CAD & GIS:  6 Ways to Automate Your Data IntegrationBridging Between CAD & GIS:  6 Ways to Automate Your Data Integration
Bridging Between CAD & GIS: 6 Ways to Automate Your Data Integrationmarketing932765
 
So einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdfSo einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdfpanagenda
 
Top 10 Hubspot Development Companies in 2024
Top 10 Hubspot Development Companies in 2024Top 10 Hubspot Development Companies in 2024
Top 10 Hubspot Development Companies in 2024TopCSSGallery
 
Varsha Sewlal- Cyber Attacks on Critical Critical Infrastructure
Varsha Sewlal- Cyber Attacks on Critical Critical InfrastructureVarsha Sewlal- Cyber Attacks on Critical Critical Infrastructure
Varsha Sewlal- Cyber Attacks on Critical Critical Infrastructureitnewsafrica
 
Tampa BSides - The No BS SOC (slides from April 6, 2024 talk)
Tampa BSides - The No BS SOC (slides from April 6, 2024 talk)Tampa BSides - The No BS SOC (slides from April 6, 2024 talk)
Tampa BSides - The No BS SOC (slides from April 6, 2024 talk)Mark Simos
 
Transcript: New from BookNet Canada for 2024: BNC SalesData and LibraryData -...
Transcript: New from BookNet Canada for 2024: BNC SalesData and LibraryData -...Transcript: New from BookNet Canada for 2024: BNC SalesData and LibraryData -...
Transcript: New from BookNet Canada for 2024: BNC SalesData and LibraryData -...BookNet Canada
 
Long journey of Ruby standard library at RubyConf AU 2024
Long journey of Ruby standard library at RubyConf AU 2024Long journey of Ruby standard library at RubyConf AU 2024
Long journey of Ruby standard library at RubyConf AU 2024Hiroshi SHIBATA
 

Kürzlich hochgeladen (20)

Emixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native developmentEmixa Mendix Meetup 11 April 2024 about Mendix Native development
Emixa Mendix Meetup 11 April 2024 about Mendix Native development
 
Digital Tools & AI in Career Development
Digital Tools & AI in Career DevelopmentDigital Tools & AI in Career Development
Digital Tools & AI in Career Development
 
A Framework for Development in the AI Age
A Framework for Development in the AI AgeA Framework for Development in the AI Age
A Framework for Development in the AI Age
 
Modern Roaming for Notes and Nomad – Cheaper Faster Better Stronger
Modern Roaming for Notes and Nomad – Cheaper Faster Better StrongerModern Roaming for Notes and Nomad – Cheaper Faster Better Stronger
Modern Roaming for Notes and Nomad – Cheaper Faster Better Stronger
 
Genislab builds better products and faster go-to-market with Lean project man...
Genislab builds better products and faster go-to-market with Lean project man...Genislab builds better products and faster go-to-market with Lean project man...
Genislab builds better products and faster go-to-market with Lean project man...
 
Design pattern talk by Kaya Weers - 2024 (v2)
Design pattern talk by Kaya Weers - 2024 (v2)Design pattern talk by Kaya Weers - 2024 (v2)
Design pattern talk by Kaya Weers - 2024 (v2)
 
Generative AI - Gitex v1Generative AI - Gitex v1.pptx
Generative AI - Gitex v1Generative AI - Gitex v1.pptxGenerative AI - Gitex v1Generative AI - Gitex v1.pptx
Generative AI - Gitex v1Generative AI - Gitex v1.pptx
 
Landscape Catalogue 2024 Australia-1.pdf
Landscape Catalogue 2024 Australia-1.pdfLandscape Catalogue 2024 Australia-1.pdf
Landscape Catalogue 2024 Australia-1.pdf
 
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
Arizona Broadband Policy Past, Present, and Future Presentation 3/25/24
 
Microsoft 365 Copilot: How to boost your productivity with AI – Part one: Ado...
Microsoft 365 Copilot: How to boost your productivity with AI – Part one: Ado...Microsoft 365 Copilot: How to boost your productivity with AI – Part one: Ado...
Microsoft 365 Copilot: How to boost your productivity with AI – Part one: Ado...
 
QCon London: Mastering long-running processes in modern architectures
QCon London: Mastering long-running processes in modern architecturesQCon London: Mastering long-running processes in modern architectures
QCon London: Mastering long-running processes in modern architectures
 
4. Cobus Valentine- Cybersecurity Threats and Solutions for the Public Sector
4. Cobus Valentine- Cybersecurity Threats and Solutions for the Public Sector4. Cobus Valentine- Cybersecurity Threats and Solutions for the Public Sector
4. Cobus Valentine- Cybersecurity Threats and Solutions for the Public Sector
 
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyesHow to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
How to Effectively Monitor SD-WAN and SASE Environments with ThousandEyes
 
Bridging Between CAD & GIS: 6 Ways to Automate Your Data Integration
Bridging Between CAD & GIS:  6 Ways to Automate Your Data IntegrationBridging Between CAD & GIS:  6 Ways to Automate Your Data Integration
Bridging Between CAD & GIS: 6 Ways to Automate Your Data Integration
 
So einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdfSo einfach geht modernes Roaming fuer Notes und Nomad.pdf
So einfach geht modernes Roaming fuer Notes und Nomad.pdf
 
Top 10 Hubspot Development Companies in 2024
Top 10 Hubspot Development Companies in 2024Top 10 Hubspot Development Companies in 2024
Top 10 Hubspot Development Companies in 2024
 
Varsha Sewlal- Cyber Attacks on Critical Critical Infrastructure
Varsha Sewlal- Cyber Attacks on Critical Critical InfrastructureVarsha Sewlal- Cyber Attacks on Critical Critical Infrastructure
Varsha Sewlal- Cyber Attacks on Critical Critical Infrastructure
 
Tampa BSides - The No BS SOC (slides from April 6, 2024 talk)
Tampa BSides - The No BS SOC (slides from April 6, 2024 talk)Tampa BSides - The No BS SOC (slides from April 6, 2024 talk)
Tampa BSides - The No BS SOC (slides from April 6, 2024 talk)
 
Transcript: New from BookNet Canada for 2024: BNC SalesData and LibraryData -...
Transcript: New from BookNet Canada for 2024: BNC SalesData and LibraryData -...Transcript: New from BookNet Canada for 2024: BNC SalesData and LibraryData -...
Transcript: New from BookNet Canada for 2024: BNC SalesData and LibraryData -...
 
Long journey of Ruby standard library at RubyConf AU 2024
Long journey of Ruby standard library at RubyConf AU 2024Long journey of Ruby standard library at RubyConf AU 2024
Long journey of Ruby standard library at RubyConf AU 2024
 

Pulsar connector on flink 1.14

  • 1. streamnative.io Pulsar Connector on Flink 1.14 The Path To Unified Batch and Streaming Yufan Sheng
  • 2. streamnative.io This document and all information and content contained herein are protected by China copyright and other intellectual property laws, and are the confidential information of StreamNative, Inc. No part of this document may be reproduced, distributed or displayed in any form or by any means, or used to make any derivative work (such as a translation or adaptation), without prior written permission from StreamNative, Inc. Copyright © 2021 StreamNative, Inc. All rights reserved.
  • 3. streamnative.io Lecturer Introduction Yufan Sheng is a software engineer at StreamNative where he devotes in flink or other streaming platform integration with Apache Pulsar. Before that he was a senior software engineer at Tencent Cloud. He was the core committer to the Barad project (Tencent Cloud Monitor Analysis Platform) and the leading engineer for Barad development at Tencent. @syhily @syhily yufan@streamnative.io
  • 4. streamnative.io Introduction to StreamNative Developed by the original creators of Apache Pulsar and Apache Bookkeeper, StreamNative provides a real-time streaming platform built for Kubernetes. StreamNative Cloud offers a turnkey platform to help you scale mission-critical applications, on enterprise-grade security and compliance.
  • 5. streamnative.io StreamNative is building a complete streaming solution With Pulsar and Flink, StreamNative offers both stream storage and stream compute for a complete streaming solution.
  • 6. streamnative.io Timeline of Pulsar connector contribution 2018/4 A simple producer & table sink FLINK-9168 2019/11 Pulsar catalog FLINK-15089 2021/1 Pulsar source on legacy SourceFunction FLINK-20727 2018/6 Pulsar producer FLINK-9641 2020/12 Announced on Flink Forward 2020 Pulsar-Flink 2.7.0 2021/8 Pulsar source on FLIP-27 FLINK-20731
  • 8. streamnative.io Batch Processing is a special case of Stream Processing A batch is just a bounded subset of stream(bounded stream).
  • 9. streamnative.io Batch vs Stream execution Stream Processor ● Bounded execution ● Unbounded execution Batch Processor ● Bounded execution ● Unbounded execution
  • 10. streamnative.io Source boundary determines Stream type That is about 60% of the truth… Source Job All source bounded => Job is bounded One source unbounded => Job is unbounded
  • 11. streamnative.io The remaining 40% of the truth … never seen this in Batch Processing, though. The (Event-time) Watermark *from the excellent Streaming 101 by Tyler Akidau: https://www.oreilly.com/ideas/the-world-beyond-batch-streaming101)
  • 12. streamnative.io Inefficiencies in Job Semantics ● Watermarks, lateness, late firing, early firing, completeness vs. latency ● Incremental results vs. definitive, final results ● Processing time As a user, you don‘t want to worry about these when executing a bounded job on the DataStream API!
  • 13. streamnative.io The remaining 40% of the truth Data is incomplete Latency SLAs Completeness and Latency is a tradeoff ETL pipelines, standing queries, anomaly detection, ML evaluation Continuous Streaming Data is as complete as it gets within the job No Low Latency SLAs ad-hoc queries, data exploration, ML training Batch Processing
  • 14. streamnative.io Flink legacy (before 1.9) API stack Streaming Runtime DataStream API DataSet API Batch Libraries Table API / SQL Streaming Libraries ● DataSet API: batch processing ● DataStream API: stream processing ● Table API/SQL: relational (batch and stream)
  • 15. streamnative.io Legacy Source Interfaces Streaming: SourceFunction Source Thread push() Batch: InputFormat Enumerate splits read splits pull() pull() JobManager
  • 16. streamnative.io Batch InputFormat Processing TaskManager TaskManager TaskManager JobManager (3) process split (1) request split (2) send split InputFormat createInputSplits(): splits assignInputSplit() openSplit(split) nextRecord(): T closeCurrentSplit()
  • 17. streamnative.io TaskManager TaskManager TaskManager JobManager (3) process split (1) request split (2) send split ● Splits are assigned to TaskManagers by the JobManager, which runs a copy of the InputFormat => Flink knows about splits and can be clever about scheduling, be reactive ● Splits can be processed in arbitrary order ● Split processing pulls records from the InputFormat ● InputFormat knows nothing about watermarks, timestamps, checkpointing => bad for streaming
  • 18. streamnative.io Stream SourceFunction Processing SourceFunction run(OutputContext) close() (1) do your thing TaskManager TaskManager TaskManager ● Source have a run-loop that they manage completely on their own ● Sources have flexibility and can efficiently work with the source system: batch accesses, dealing with multiple topics from one consumer, threading model, etc… ● Flink does not know what’s going on inside and can’t be clever about it ● Sources have to implement their own per-partition watermarking, idleness tracking, what have you
  • 19. streamnative.io A New (unified) Source Interface (FLIP-27) Source createSplitEnumerator() createSplitReader() SplitEnumerator discoverNewSplits() nextSplit() isDone() snapshotState() SplitReader addSplit() hasAvailable(): Future emitNext(Context): Status snapshotState() ● This must support both batch and streaming use cases, allow Flink to be clever, be able to deal with event-time, watermarks, source idiosyncrasies, and enable snapshotting. ● This should enable new features: generic idleness detection, event-time alignment.* * FLINK-10886: Event-time alignment for sources
  • 20. streamnative.io Execution Style TaskManager TaskManager TaskManager JobManager (3) process split (1) request split (2) send split
  • 23. streamnative.io Pulsar Subscription Modes Different subscription modes have different semantics: Exclusive/Failover - guaranteed order, single active consumer Shared - multiple active consumers, no order Key_Shared - multiple active consumers, order for given key Subscription D Consumer D-1 Consumer D-2 Key-Shared < K 1, V 10 > < K 1, V 11 > < K 1, V 12 > < K 2 ,V 2 0 > < K 2 ,V 2 1> < K 2 ,V 2 2 > Subscription C Consumer C-1 Consumer C-2 Shared < K 1, V 10 > < K 2 ,V 2 1> < K 1, V 12 > < K 2 ,V 2 0 > < K 1, V 11 > < K 2 ,V 2 2 > Subscription A Consumer A Exclusive Subscription B Consumer B-1 Consumer B-2 In case of failure in Consumer B-1 Failover Pulsar Partition is also a sub set of topics which has only one partition and could be consumed directly. Producer 1 Producer 2 Pulsar Topic m0 m1 m2 m3 m4
  • 24. streamnative.io Pulsar Split design PulsarPartitionSplit TopicPartition partition StartCursor startCursor StopCursor stopCursor MessageId latestConsumedId TxnID uncommittedTransactionId TopicPartition String topic int partitionId TopicRange range wrap into 1 to 1 Partitioned Topic Non-Partitioned Topic 1 to n 1 to 1 For Exclusive, Failover and Shared Failover Consumer B-1 Consumer B-0 Subscription B m1 m2 m3 m4 m0 In case of failure in Consumer B-0 Consumer A-1 Consumer A-0 Subscription A m1 m2 m3 m4 m0 Exclusive X Consumer C-1 Consumer C-2 Consumer C-3 Subscription C Shared < k 1 , v 0 > < k 1 , v 4 > < k 3 , v 2 > <k 2, v1 > < k 2 , v 3 >
  • 25. streamnative.io Pulsar Split design For Key_Shared TopicPartition String topic int partitionId Partitioned Topic Non-Partitioned Topic 1 to n 1 to 1 RangeGenerator TopicPartition String topic int partitionId TopicRange range 1 to n … Shared and Key_Shared can share the same subscription for all consumers, why we create the split on partition? Consumer D-1 Consumer D-2 Consumer D-3 Subscription D < k 2 , v 1 > < k 2 , v 3 > <k3 ,v2 > < k 1 , v 0 > < k 1 , v 4 > Key-Shared
  • 26. streamnative.io Pulsar Split design Create the split on partition Consumer.seek() can only be executed on single topic with one partition or no partition.
  • 27. streamnative.io Pulsar Enumerator design Pull splits Enumerator TopicListSubscriber TopicPatternSubscriber Continuous unbound (Stream) would cycle query Bounded (Batch) would query only once
  • 28. streamnative.io Pulsar Enumerator design Assign split on Exclusive, Failover and Key_Shared subscription Enumerator Reader 1 Reader 2 Reader 3 Split 3 Split 4 Split 7 Split 2 Split 9 Split 6 Split 1 Split 5 Split 8 Consumer D-1 Consumer D-2 Consumer D-3 Subscription D < k 2 , v 1 > < k 2 , v 3 > <k3 ,v2 > < k 1 , v 0 > < k 1 , v 4 > Key-Shared Failover Consumer B-1 Consumer B-0 Subscription B m1 m2 m3 m4 m0 In case of failure in Consumer B-0 Consumer A-1 Consumer A-0 Subscription A m1 m2 m3 m4 m0 Exclusive X
  • 29. streamnative.io Pulsar Enumerator design Assign split on Shared subscription Enumerator Reader 1 Reader 2 Reader 3 Split 1 Split 2 Split 3 Split 1 Split 3 Split 2 Split 1 Split 2 Split 3 Why don’t share splits for Key_Shared subscription. Consumer C-1 Consumer C-2 Consumer C-3 Subscription C Shared < k 1 , v 0 > < k 1 , v 4 > < k 3 , v 2 > <k 2, v1 > < k 2 , v 3 >
  • 30. streamnative.io Pulsar Reader design Ordered message consuming (Failover, Exclusive) Ordered Reader Ordered Split Reader SortedMap<Long, Map<TopicPartition, MessageId>> cursorsToCommit ConcurrentMap<TopicPartition, MessageId> cursorsOfFinishedSplits ScheduledExecutorService cursorScheduler Used cumulative message acknowledge, we only memorize last consumed message id. Acknowledge message with id when flink finished the checkpointing. Failover Consumer B-1 Consumer B-0 Subscription B m1 m2 m3 m4 m0 In case of failure in Consumer B-0 Consumer A-1 Consumer A-0 Subscription A m1 m2 m3 m4 m0 Exclusive X
  • 31. streamnative.io Pulsar Reader design Unordered message consuming (Shared, Key_Shared) Unordered Reader Unordered Split Reader TransactionCoordinatorClient coordinatorClient SortedMap<Long, List<TxnID>> transactionsToCommit List<TxnID> transactionsOfFinishedSplits Acknowledge message in a transaction, we only hold one transaction at a time. Commit the transaction when flink finished the checkpointing. Consumer D-1 Consumer D-2 Consumer D-3 Subscription D < k 2 , v 1 > < k 2 , v 3 > <k3 ,v2 > < k 1 , v 0 > < k 1 , v 4 > Key-Shared Consumer C-1 Consumer C-2 Consumer C-3 Subscription C Shared < k 1 , v 0 > < k 1 , v 4 > < k 3 , v 2 > <k 2, v1 > < k 2 , v 3 >
  • 32. streamnative.io Flink type system vs Pulsar Schema Pulsar ● Schema: Client side, used for serialize and deserialize message. ● SchemaInfo: Server and Client side, used for compatible validation and schema evolution. ● SchemaDefinition: Client side, used for creating a Pulsar Schema. Flink ● (De)serializationSchema: Operator side, used for deserialize source element and provide a TypeInformation. ● TypeInfomation: Shared among TaskManagers, used for message serialization and transportation. The flink’s default type systems, would fallback to Kyro when no types provided. ● TypeSerializer: Used for message serialize and deserialize.
  • 33. streamnative.io Pulsar Schema on Flink Schema.BYTES Reader 1 DeserializationSchema<T> Downstr eam Schema.BYTES Reader 1 Schema<T> TypeInformation<T> Downstr eam
  • 34. streamnative.io Pulsar Schema evolution on Flink Schema<T> Reader 1 TypeInformation<T> Downstr eam Make Pulsar Schema serializable. Auto mapping Schema to Flink TypeInformation.
  • 35. streamnative.io Checkpoint on Pulsar Source Job Coordinator Enumerator Ordered Reader Ordered Split Reader Unordered Reader Unordered Split Reader 1. Triggered a checkpoint 2. Snapshot the split assign status. 1. Triggered a checkpoint 2. Snapshot the consuming position. 3. Save position with checkpoint id. 4. Notify checkpoint complete 5. Acknowledge message id to pulsar 1. Triggered a checkpoint 2. Start a new transaction. 3. Save transaction id with checkpoint id. 4. Notify checkpoint com plete 5. Com m it transaction
  • 36. streamnative.io Pulsar Source configuration We have exposed all the configuration options in: • PulsarSourceOptions: Pulsar Source and Pulsar Consumer config option. • PulsarOptions: Options for PulsarAdmin and PulsarClient builder. These options are define by flink’s ConfigOption, with strong type support.
  • 37. streamnative.io Overview of PulsarSourceBuilder PulsarSource<String> source = PulsarSource .builder() .setServiceUrl(PULSAR_BROKER_URL) .setAdminUrl(PULSAR_BROKER_HTTP_URL) .setSubscriptionName("flink-source-1") .setTopics(Arrays.asList(TOPIC1, TOPIC2)) .setDeserializationSchema(PulsarDeserializationSchema.flinkSchema(new SimpleStringSchema())) .setUnbounded(StopCursor.atEventTime(System.currentTimeMillis())) .build(); A simple sample on consuming string messages from Pulsar
  • 38. streamnative.io You should provide at least these information. setServiceUrl setAdminUrl setSubscriptionName setTopics/setTopicPattern setDeserializationSchema Overview of PulsarSourceBuilder
  • 39. streamnative.io Create PulsarDeserializationSchema You can create a PulsarDeserializationSchema in three ways: 1. Using a Pulsar Schema, we would auto create it’s related flink TypeInformation. 2. Using a Flink DeserializationSchema, we would use pure flink mechanism. 3. Using a Flink TypeInformation, it’s not recommend for it doesn’t compatible with others above.
  • 40. streamnative.io Provide StartCursor and StopCursor StartCursor could be created in four ways. 1. earliest(): From the earliest available message in the topic. 2. latest(): From the latest available message in the topic. 3. fromMessageId(MessageId messageId, boolean inclusive): From a specified message id. 4. fromMessageTime(long timestamp): From a specified message time. Caution: 1. If the provided message id/time is bigger than the available message, we would start consuming from the latest(). 2. If the provided message id/time doesn’t existed (deleted), we would find the next available message, it may fallback to the latest() finally.
  • 41. streamnative.io Provide StartCursor and StopCursor StopCursor could be created in five ways. 1. never(): We never stop, the source is infinite. 2. latest(): Query the latest message and stop after this message id. 3. atMessageId(MessageId messageId): Stop at a specified message id (exclude). 4. afterMessageId(MessageId messageId): Stop after a specified message id (include). 5. atEventTime(long timestamp): Stop at a specified message time (exclude). Caution: latest() is used for test and batch data consuming. You should fully understand its internal logic before using.