SlideShare a Scribd company logo
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

More Related Content

PDF
Apache Flink Stream Processing
PDF
Unified Stream and Batch Processing with Apache Flink
PDF
Marton Balassi – Stateful Stream Processing
PDF
Tech Talk @ Google on Flink Fault Tolerance and HA
PPTX
Apache Flink Training: System Overview
PDF
Stateful stream processing with Apache Flink
PDF
Apache Flink Internals: Stream & Batch Processing in One System – Apache Flin...
PDF
Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)
Apache Flink Stream Processing
Unified Stream and Batch Processing with Apache Flink
Marton Balassi – Stateful Stream Processing
Tech Talk @ Google on Flink Fault Tolerance and HA
Apache Flink Training: System Overview
Stateful stream processing with Apache Flink
Apache Flink Internals: Stream & Batch Processing in One System – Apache Flin...
Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)

What's hot (20)

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

Similar to Pulsar connector on flink 1.14 (20)

PDF
Ai dev world utilizing apache pulsar, apache ni fi and minifi for edgeai io...
PDF
Apache Flink Training Workshop @ HadoopCon2016 - #1 System Overview
PPTX
Apache Kafka 101 by Confluent Developer Friendly
PDF
FLiP Into Trino
PDF
The Beam Vision for Portability: "Write once run anywhere"
PPTX
apache-kafka-101 a simple presentation on how to use Kafka
PPTX
Talk Python To Me: Stream Processing in your favourite Language with Beam on ...
PDF
Flink Forward Berlin 2018: Thomas Weise & Aljoscha Krettek - "Python Streamin...
PPTX
Python Streaming Pipelines with Beam on Flink
PDF
Fault Tolerance at Speed
PDF
Flink Forward Berlin 2017: Aljoscha Krettek - Talk Python to me: Stream Proce...
ODP
Real time-collaborative-editor-presentation
PPTX
Advanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 Keynote
PDF
Explore Android Internals
PDF
Project Reactor Now and Tomorrow
PDF
Streaming Dataflow with Apache Flink
PDF
Near real-time anomaly detection at Lyft
PPTX
Counting Elements in Streams
PDF
Tornado Web Server Internals
PDF
BigQuery case study in Groovenauts & Dive into the DataflowJavaSDK
Ai dev world utilizing apache pulsar, apache ni fi and minifi for edgeai io...
Apache Flink Training Workshop @ HadoopCon2016 - #1 System Overview
Apache Kafka 101 by Confluent Developer Friendly
FLiP Into Trino
The Beam Vision for Portability: "Write once run anywhere"
apache-kafka-101 a simple presentation on how to use Kafka
Talk Python To Me: Stream Processing in your favourite Language with Beam on ...
Flink Forward Berlin 2018: Thomas Weise & Aljoscha Krettek - "Python Streamin...
Python Streaming Pipelines with Beam on Flink
Fault Tolerance at Speed
Flink Forward Berlin 2017: Aljoscha Krettek - Talk Python to me: Stream Proce...
Real time-collaborative-editor-presentation
Advanced Stream Processing with Flink and Pulsar - Pulsar Summit NA 2021 Keynote
Explore Android Internals
Project Reactor Now and Tomorrow
Streaming Dataflow with Apache Flink
Near real-time anomaly detection at Lyft
Counting Elements in Streams
Tornado Web Server Internals
BigQuery case study in Groovenauts & Dive into the DataflowJavaSDK
Ad

Recently uploaded (20)

PPTX
OMC Textile Division Presentation 2021.pptx
PPTX
1. Introduction to Computer Programming.pptx
PPTX
A Presentation on Artificial Intelligence
PDF
Video forgery: An extensive analysis of inter-and intra-frame manipulation al...
PDF
A comparative study of natural language inference in Swahili using monolingua...
PPTX
cloud_computing_Infrastucture_as_cloud_p
PDF
Getting Started with Data Integration: FME Form 101
PDF
Mobile App Security Testing_ A Comprehensive Guide.pdf
PDF
Per capita expenditure prediction using model stacking based on satellite ima...
PDF
Build a system with the filesystem maintained by OSTree @ COSCUP 2025
PDF
Unlocking AI with Model Context Protocol (MCP)
PDF
A comparative analysis of optical character recognition models for extracting...
PPTX
Machine Learning_overview_presentation.pptx
PDF
7 ChatGPT Prompts to Help You Define Your Ideal Customer Profile.pdf
PDF
Building Integrated photovoltaic BIPV_UPV.pdf
PPTX
Digital-Transformation-Roadmap-for-Companies.pptx
PDF
Encapsulation_ Review paper, used for researhc scholars
PDF
TokAI - TikTok AI Agent : The First AI Application That Analyzes 10,000+ Vira...
PDF
Architecting across the Boundaries of two Complex Domains - Healthcare & Tech...
PDF
Network Security Unit 5.pdf for BCA BBA.
OMC Textile Division Presentation 2021.pptx
1. Introduction to Computer Programming.pptx
A Presentation on Artificial Intelligence
Video forgery: An extensive analysis of inter-and intra-frame manipulation al...
A comparative study of natural language inference in Swahili using monolingua...
cloud_computing_Infrastucture_as_cloud_p
Getting Started with Data Integration: FME Form 101
Mobile App Security Testing_ A Comprehensive Guide.pdf
Per capita expenditure prediction using model stacking based on satellite ima...
Build a system with the filesystem maintained by OSTree @ COSCUP 2025
Unlocking AI with Model Context Protocol (MCP)
A comparative analysis of optical character recognition models for extracting...
Machine Learning_overview_presentation.pptx
7 ChatGPT Prompts to Help You Define Your Ideal Customer Profile.pdf
Building Integrated photovoltaic BIPV_UPV.pdf
Digital-Transformation-Roadmap-for-Companies.pptx
Encapsulation_ Review paper, used for researhc scholars
TokAI - TikTok AI Agent : The First AI Application That Analyzes 10,000+ Vira...
Architecting across the Boundaries of two Complex Domains - Healthcare & Tech...
Network Security Unit 5.pdf for BCA BBA.

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.