SlideShare a Scribd company logo
1 of 62
Download to read offline
Pulsar Summit
San Francisco
Hotel Nikko
August 18 2022
Tech Deep Dive
Message Redelivery:
An Unexpected
Journey
David Kjerrumgaard
Developer Advocate • StreamNative
• Apache Pulsar Committer & author of
Pulsar In Action
• Former Principal Software Engineer on
Splunk’s Pulsar-as-a-Service platform
• Over 15 years experience in Big
Data/Streaming
David Kjerrumgaard
Developer Advocate
StreamNative
Pulsar Client & Broker
streamnative.io
10,000 Foot Level
● Pulsar Clients and Brokers interact with
one another by exchanging Command
Messages back-and-forth.
● Brokers are continually listening for
incoming commands on a known port.
● Clients create a connection to the Broker
on this port and start issuing commands.
streamnative.io
Command Categories
● These commands can be broken down into different categories
based on how they are used.
streamnative.io
Client Commands
● These commands are used by the
Pulsar Client & Broker to establish
communication, authenticate, and
perform topic lookups.
● Commands are used to transition the
client through various states, e.g.,
disconnected to connected.
Client Creation
● When the build() method is called, under the covers
the CONNECT command is sent to the address
specified by the serviceUrl property.
● If the client is authenticated, then the Pulsar Broker
will respond with a CONNECTED command.
streamnative.io
Client Creation Flow
● The builder dispatches a
CONNECT command
(including credentials) to
establish a connection.
● The Broker authenticates
the client and sends a
CONNECTED command to
indicate success.
Message
Publication
streamnative.io
Producer Commands
● These commands are used to move
messages from the Producer to the
Broker.
● Commands are used to perform
message handshaking between the
Producer and the Broker, e.g.,
SEND/SEND_RECEIPT
Producer Creation
● When the create() method is called, under the covers a
PRODUCER command is sent to the Broker.
● If the producer is authorized and has a compatible
schema, then the Pulsar Broker will respond with a
PRODUCER_SUCCESS command.
Producer Creation – Configuration
ProducerBuilder Method Description
enableBatching (boolean enable) Control whether automatic batching of
messages is enabled for the producer.
batchingMaxBytes (int max) Set the maximum number of bytes
permitted in a batch
batchingMaxMessages (int max) Set the maximum number of messages
permitted in a batch.
batchingMaxPublishDelay (long delay,
TimeUnit timeUnit)
Set the time period within which the
messages sent will be batched
maxPendingMessages (int max) Set the max size of the queue holding the
messages pending to receive an
acknowledgment from the broker.
sendTimeout(int timeout, TimeUnit
unit)
If a message is not acknowledged by the
server before the sendTimeout expires, an
error will be reported.
streamnative.io
Producer Creation Flow
● The PRODUCER command
includes additional info such
as the Schema, etc.
● The Broker creates a
Producer object on the
Broker side and sends a
PRODUCER_SUCCESS
command if successful.
Broker-Side Producer Object
● An org.apache.pulsar.broker.service.Producer
object is created on the broker-side that retains a
reference to the corresponding Topic object.
● The Topic object is responsible for writing all the
received messages to the Managed Ledger.
Client-Side Producer Object
● A Producer object is created on the client-side that has a
BatchMessageContainer object that is used to retain
messages until the batch is considered full.
● Messages published via the send()/sendAsync()
methods are stored in the BatchMessageContainer.
● When the batch is full, messages are removed from the
BatchMessageContainer and moved to a
pendingMessages queue to await acknowledgement.
Message Publication
● The producing application
can call the send()
method multiple times.
● Each of these messages
are added to the
BatchMessageContainer
until a batch send is
triggered due to size or
time.
Message Publication – cont.
● Once the batch is considered
“full”, the raw messages are
prepared for transmission to the
Broker.
● Each message is annotated with
metadata such as the
producer's name, sequenceID,
and creation time.
Message Publication – cont.
● The wrapped messages are added to the pendingMessage
queue and sent to the Broker via a SEND command.
Message Publication – Success.
● Once the message is persisted to BookKeeper the Broker
notifies the Producer via a SEND_RECEIPT command and the
message is removed from the pendingMessage queue.
Publication Redelivery Scenario #1
● If the message cannot be persisted, the Broker notifies the
Producer via a SEND_ERROR command and the message will
be resent from the pendingMessage queue.
Publication Redelivery Scenario #1 – cont
● If the PRODUCER receives a SEND_ERROR message, then it
will resend all the messages in the pendingMessages queue.
Publication Redelivery Scenario #2
● If there is an issue during the transmission of the message,
then it remains in the pendingMessages queue and will be
resent will the Producer reconnects.
Publication Redelivery Scenario #2
● After the Producer
reconnects, all the
messages in the
pendingMessages
queue are re-sent
automatically.
Message
Consumption
streamnative.io
Consumer Commands
● These commands are used to
move messages from the
Broker to the Consumer.
● Commands are used to
perform message
handshaking between the
Consumer and the Broker,
e.g., MESSAGE/ACK
Consumer Creation
● When the subscribe() method is called, under the
covers a SUBSCRIBE command is sent to the Broker.
● If the Consumer is authorized and has specified a
compatible schema, then the Pulsar Broker will respond
with a SUCCESS command.
Consumer Creation – Configuration
ConsumerBuilder Method Description
acknowledgmentGroupTime (long delay,
TimeUnit unit)
Group the consumer acknowledgments for the
specified time.
ackTimeout(long ackTimeout,
TimeUnit timeUnit)
Set the timeout for unacked messages
ackTimeoutTickTime (long tickTime,
TimeUnit timeUnit)
Define the granularity of the ack-timeout
redelivery.
ackTimeoutRedeliveryBackoff (Redeli
veryBackoff backoff)
Allows you to specify a custom redelivery
strategy for message that have exceeded the
ack timeout
negativeAckRedeliveryDelay (long
delay, TimeUnit timeUnit)
Set the delay to wait before re-delivering
messages that have failed to be process.
negativeAckRedeliveryBackoff (Redel
iveryBackoff backoff)
Allows you to specify a custom redelivery
strategy for negatively acknowledged messages
streamnative.io
Consumer Creation Flow
● The SUBSCRIBE command
includes additional info such
as the Schema, etc.
● The Broker creates a
Consumer object on the
Broker side and sends a
SUCCESS command if
successful.
Broker-Side Consumer Object
Client-Side Consumer Object
The Message
Delivery
Process
Message Flow
● A pull-based model controlled by the FLOW command.
Message Flow - Backend
● The messages are chosen by the Subscription and read from
BookKeeper.
Message Flow – cont.
● The messages are added to the pendingAcks HashMap
before being sent back via individual MESSAGE commands.
Message Consumption
● When you consume messages using the receive() /
receiveAsync() methods, the messages are taken from the
incomingMessages queue and given to the application.
Automated Message Flow
● The client-side Consumer periodically requests new
messages from the broker-side Consumer (pull-based)
● This process is triggered automatically when the
client-side Consumer’s incomingMessages queue drops
below 50% of its capacity.
● When this condition is detected a FLOW command is
dispatched to the broker-side Consumer to request
another batch of messages.
Post-Processing Options
● After the message is processed, it can either be
acknowledged, negatively acknowledged, or sent to the
retry letter topic to be re-delivered after a specified length
of time.
The Message
Acknowledgement
Process
Message Acknowledgement
● After a message has been successfully consumed, it must
be acknowledged to prevent redelivery/reprocessing.
● The behavior of the Acknowledgment process can be
controlled by the following settings in the ConsumerBuilder
• acknowledgmentGroupTime(long delay, TimeUnit unit)
• ackTimeout(long ackTimeout, TimeUnit timeUnit)
• ackTimeoutRedeliveryBackoff(RedeliveryBackoff ackBackoff)
• ackTimeoutTickTime(long tickTime, TimeUnit timeUnit)
Message Acknowledgement
● If the acknowledgmentGroupTime property is set, then the acks will
be grouped together for the specified interval before being sent
together, otherwise an ack is sent immediately.
Message Acknowledgement - Broker
● On the Broker side, the ack is removed from the pendingAcks list, and the
subscription updates the cursor position. An ACK_RESPONSE command is
sent back to the client to complete the handshake.
Redelivery of
Messages Due to
Ack Timeout
Waiting
● On the client-side, the UnackedMessageTracker tracks the
receipt of the ACK commands that were sent to the Broker to
ensure that the subscription cursor is positioned correctly.
● Any ACKs that do not receive a corresponding ACK_RESPONSE
command must be resent to complete the handshake. Missing
ACK_RESPONSEs indicate the possibility of a network outage, etc.
● How long the client decided to wait for these ACK_RESPONSEs is
referred to as the “Ack Timeout”
The UnackedMessageTracker
● Messages are added to the UnackedMessageTracker as
their corresponding ACKs are sent to the Broker.
The UnackedMessageTracker
● To ensure that we only redeliver messages that have not
been ACKed by the Broker, we remove message as their
corresponding ACK_RESPONSEs are received from the
Broker.
The UnackedMessageTracker
● There are 3 variations of the UnackedMessageTracker, and
the one that is used is determined by the way that the
consumer was configured.
Ack Timeout Configuration
● If the ackTimeout property is set to zero, then ACK tracking
is disabled, and we have AT_MOST_ONCE processing
semantics.
● Otherwise, we use one of the two timer-based variations
that waits a period for the ACK_RESPONSEs before
resending the ACKs to the Broker.
● The only difference between the two is how the ack timeout
interval is calculated.
Timer-Based UnackedMessgeTrackers
● Internally, both timer-based UnackedMessageTracker versions
use a Timer task to trigger the redelivery of the ACKs.
● The base UnackedMessageTracker operates on a fixed interval
equal to either the ackTimeoutTickTime property (if set) or else
the ackTimeout property.
● The UnackedMessageRedeliveryTracker operates on a variable
interval based upon the value of the
ackTimeoutRedeliveryBackoff property.
Timer-Based UnackedMessgeTrackers
● Here is an example of an UnackedMessageRedeliveryTracker
configured to use an Exponential backoff strategy.
Ack Timeout
● Once the ack timeout interval has elapsed, the consumer
sends a REDELIVER_UNACKNOWLEDGED_MESSAGES
command to the Broker.
Ack Timeout
● The REDELIVER_UNACKNOWLEDGED_MESSAGES triggers
the Broker-side consumer to resend ALL the messages in
PendingAcks.
Redelivery of
Negatively Acked
Messages
The NegativeAcksTracker
● Messages are added to the NegativeAcksTracker when
the application calls consumer.negativeAck(msg)
Negative Acks Configuration
● There is only one variation of the
NegativeAcksTracker, and it uses a Timer task to
trigger the redelivery of the negatively acked
messages.
● The Timer operates on a fixed interval if the
negativeAckRedeliveryDelay property is set.
Otherwise, it uses the strategy specified by the
negativeAckRedeliveryBackoff property.
Timeout
● Once the redelivery delay has elapsed, the consumer
sends a REDELIVER_UNACKNOWLEDGED_MESSAGES
command to the Broker.
Redelivery of
Delayed
Messages
Delayed Message Delivery
● If the enableRetry property is set to true, the consumer
can request that a message be re-delivered after a specified
delay, e.g., consumer.reconsumeLater(msg, 3,
TimeUnit.SECONDS)
● Under the covers, this process utilizes the
deliverAfter(delayTime, unit) method of the
Producer class.
Delayed Message Delivery
● When you first call the reconsumeLater method on the
consumer, a Producer object is initialized using the following
code block.
Delayed Message Delivery
● Next, we check to see how many times we have already
tried to redeliver this message by examining the
SYSTEM_PROPERTY_RECONSUMETIMES property in the
Message’s properties.
● If that value exceeds the maxRedeliverCount setting of the
DLQ policy, then the message is routed to the DLQ.
● Otherwise, it is scheduled for delayed deliver via a call to
producer.newMessage().deliverAfter(delayTime,
unit);
streamnative.io
Key Takeaways
➔ Message redelivery can occur
when Producing or Consuming
messages in Pulsar
➔ When producing, messages get
re-delivered if the Broker doesn’t
acknowledge receipt of the
message in a timely manner.
➔ When consuming, messages get
re-delivered under 3 different
circumstances. Ack timeout,
negative ack, and delay
David Kjerrumgaard
Thank you!
david@streamnative.io
@DavidKjerrumga1
Pulsar Summit
San Francisco
Hotel Nikko
August 18 2022

More Related Content

What's hot

Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
SANG WON PARK
 

What's hot (20)

Grafana.pptx
Grafana.pptxGrafana.pptx
Grafana.pptx
 
Introduction to Kafka Cruise Control
Introduction to Kafka Cruise ControlIntroduction to Kafka Cruise Control
Introduction to Kafka Cruise Control
 
Apache Kafka
Apache KafkaApache Kafka
Apache Kafka
 
APACHE KAFKA / Kafka Connect / Kafka Streams
APACHE KAFKA / Kafka Connect / Kafka StreamsAPACHE KAFKA / Kafka Connect / Kafka Streams
APACHE KAFKA / Kafka Connect / Kafka Streams
 
When apache pulsar meets apache flink
When apache pulsar meets apache flinkWhen apache pulsar meets apache flink
When apache pulsar meets apache flink
 
kafka
kafkakafka
kafka
 
Distributed tracing 101
Distributed tracing 101Distributed tracing 101
Distributed tracing 101
 
Apache Kafka
Apache KafkaApache Kafka
Apache Kafka
 
Kafka Tutorial - Introduction to Apache Kafka (Part 1)
Kafka Tutorial - Introduction to Apache Kafka (Part 1)Kafka Tutorial - Introduction to Apache Kafka (Part 1)
Kafka Tutorial - Introduction to Apache Kafka (Part 1)
 
Introduction To RabbitMQ
Introduction To RabbitMQIntroduction To RabbitMQ
Introduction To RabbitMQ
 
Introduction to Apache Kafka
Introduction to Apache KafkaIntroduction to Apache Kafka
Introduction to Apache Kafka
 
CDC patterns in Apache Kafka®
CDC patterns in Apache Kafka®CDC patterns in Apache Kafka®
CDC patterns in Apache Kafka®
 
Apache Kafka Architecture & Fundamentals Explained
Apache Kafka Architecture & Fundamentals ExplainedApache Kafka Architecture & Fundamentals Explained
Apache Kafka Architecture & Fundamentals Explained
 
Grafana introduction
Grafana introductionGrafana introduction
Grafana introduction
 
RabbitMQ
RabbitMQ RabbitMQ
RabbitMQ
 
Introduction to Kafka Streams
Introduction to Kafka StreamsIntroduction to Kafka Streams
Introduction to Kafka Streams
 
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
 
Introduction to apache kafka
Introduction to apache kafkaIntroduction to apache kafka
Introduction to apache kafka
 
Kafka presentation
Kafka presentationKafka presentation
Kafka presentation
 
An Introduction to Apache Kafka
An Introduction to Apache KafkaAn Introduction to Apache Kafka
An Introduction to Apache Kafka
 

Similar to Message Redelivery: An Unexpected Journey - Pulsar Summit SF 2022

1 messagepassing-121015032028-phpapp01
1 messagepassing-121015032028-phpapp011 messagepassing-121015032028-phpapp01
1 messagepassing-121015032028-phpapp01
Zaigham Abbas
 
Productivity Acceleration Tools for SOA Testers
Productivity Acceleration Tools for SOA TestersProductivity Acceleration Tools for SOA Testers
Productivity Acceleration Tools for SOA Testers
WSO2
 

Similar to Message Redelivery: An Unexpected Journey - Pulsar Summit SF 2022 (20)

Introduction to Kafka and Event-Driven
Introduction to Kafka and Event-DrivenIntroduction to Kafka and Event-Driven
Introduction to Kafka and Event-Driven
 
Introduction to Kafka and Event-Driven
Introduction to Kafka and Event-DrivenIntroduction to Kafka and Event-Driven
Introduction to Kafka and Event-Driven
 
MQTT
MQTTMQTT
MQTT
 
HHM-3540: The IBM MQ Light API: From Developer Laptop to Enterprise Data Cen...
 HHM-3540: The IBM MQ Light API: From Developer Laptop to Enterprise Data Cen... HHM-3540: The IBM MQ Light API: From Developer Laptop to Enterprise Data Cen...
HHM-3540: The IBM MQ Light API: From Developer Laptop to Enterprise Data Cen...
 
Introduction to MQ Telemetry Transport (MQTT)
Introduction to MQ Telemetry Transport (MQTT)Introduction to MQ Telemetry Transport (MQTT)
Introduction to MQ Telemetry Transport (MQTT)
 
Reliable System Integration and Scaling with WSO2 Message Broker
Reliable System Integration and Scaling with WSO2 Message BrokerReliable System Integration and Scaling with WSO2 Message Broker
Reliable System Integration and Scaling with WSO2 Message Broker
 
message passing
 message passing message passing
message passing
 
1 messagepassing-121015032028-phpapp01
1 messagepassing-121015032028-phpapp011 messagepassing-121015032028-phpapp01
1 messagepassing-121015032028-phpapp01
 
MoP(MQTT on Pulsar) - a Powerful Tool for Apache Pulsar in IoT - Pulsar Summi...
MoP(MQTT on Pulsar) - a Powerful Tool for Apache Pulsar in IoT - Pulsar Summi...MoP(MQTT on Pulsar) - a Powerful Tool for Apache Pulsar in IoT - Pulsar Summi...
MoP(MQTT on Pulsar) - a Powerful Tool for Apache Pulsar in IoT - Pulsar Summi...
 
Kafka aws
Kafka awsKafka aws
Kafka aws
 
IBM IMPACT 2014 - AMC-1882 Building a Scalable & Continuously Available IBM M...
IBM IMPACT 2014 - AMC-1882 Building a Scalable & Continuously Available IBM M...IBM IMPACT 2014 - AMC-1882 Building a Scalable & Continuously Available IBM M...
IBM IMPACT 2014 - AMC-1882 Building a Scalable & Continuously Available IBM M...
 
Introduction to MicroProfile Metrics
Introduction to MicroProfile MetricsIntroduction to MicroProfile Metrics
Introduction to MicroProfile Metrics
 
Productivity Acceleration Tools for SOA Testers
Productivity Acceleration Tools for SOA TestersProductivity Acceleration Tools for SOA Testers
Productivity Acceleration Tools for SOA Testers
 
DataPower-MQ Integration Deep Dive
DataPower-MQ Integration Deep DiveDataPower-MQ Integration Deep Dive
DataPower-MQ Integration Deep Dive
 
Troubleshooting and Best Practices with WSO2 Enterprise Integrator
Troubleshooting and Best Practices with WSO2 Enterprise IntegratorTroubleshooting and Best Practices with WSO2 Enterprise Integrator
Troubleshooting and Best Practices with WSO2 Enterprise Integrator
 
Mumbai MuleSoft Meetup #20
Mumbai MuleSoft Meetup #20Mumbai MuleSoft Meetup #20
Mumbai MuleSoft Meetup #20
 
mqtt intro short
mqtt intro shortmqtt intro short
mqtt intro short
 
1463401 rc214-mqtt-update
1463401 rc214-mqtt-update1463401 rc214-mqtt-update
1463401 rc214-mqtt-update
 
Message queuing telemetry transport (mqtt) id and other type parameters
Message queuing telemetry transport (mqtt) id and other type parametersMessage queuing telemetry transport (mqtt) id and other type parameters
Message queuing telemetry transport (mqtt) id and other type parameters
 
Writing Networking Clients in Go - GopherCon 2017 talk
Writing Networking Clients in Go - GopherCon 2017 talkWriting Networking Clients in Go - GopherCon 2017 talk
Writing Networking Clients in Go - GopherCon 2017 talk
 

More from StreamNative

Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022
Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022
Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022
StreamNative
 

More from StreamNative (20)

Is Using KoP (Kafka-on-Pulsar) a Good Idea? - Pulsar Summit SF 2022
Is Using KoP (Kafka-on-Pulsar) a Good Idea? - Pulsar Summit SF 2022Is Using KoP (Kafka-on-Pulsar) a Good Idea? - Pulsar Summit SF 2022
Is Using KoP (Kafka-on-Pulsar) a Good Idea? - Pulsar Summit SF 2022
 
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
 
Blue-green deploys with Pulsar & Envoy in an event-driven microservice ecosys...
Blue-green deploys with Pulsar & Envoy in an event-driven microservice ecosys...Blue-green deploys with Pulsar & Envoy in an event-driven microservice ecosys...
Blue-green deploys with Pulsar & Envoy in an event-driven microservice ecosys...
 
Distributed Database Design Decisions to Support High Performance Event Strea...
Distributed Database Design Decisions to Support High Performance Event Strea...Distributed Database Design Decisions to Support High Performance Event Strea...
Distributed Database Design Decisions to Support High Performance Event Strea...
 
Simplify Pulsar Functions Development with SQL - Pulsar Summit SF 2022
Simplify Pulsar Functions Development with SQL - Pulsar Summit SF 2022Simplify Pulsar Functions Development with SQL - Pulsar Summit SF 2022
Simplify Pulsar Functions Development with SQL - Pulsar Summit SF 2022
 
Towards a ZooKeeper-less Pulsar, etcd, etcd, etcd. - Pulsar Summit SF 2022
Towards a ZooKeeper-less Pulsar, etcd, etcd, etcd. - Pulsar Summit SF 2022Towards a ZooKeeper-less Pulsar, etcd, etcd, etcd. - Pulsar Summit SF 2022
Towards a ZooKeeper-less Pulsar, etcd, etcd, etcd. - Pulsar Summit SF 2022
 
Validating Apache Pulsar’s Behavior under Failure Conditions - Pulsar Summit ...
Validating Apache Pulsar’s Behavior under Failure Conditions - Pulsar Summit ...Validating Apache Pulsar’s Behavior under Failure Conditions - Pulsar Summit ...
Validating Apache Pulsar’s Behavior under Failure Conditions - Pulsar Summit ...
 
Cross the Streams! Creating Streaming Data Pipelines with Apache Flink + Apac...
Cross the Streams! Creating Streaming Data Pipelines with Apache Flink + Apac...Cross the Streams! Creating Streaming Data Pipelines with Apache Flink + Apac...
Cross the Streams! Creating Streaming Data Pipelines with Apache Flink + Apac...
 
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
Unlocking the Power of Lakehouse Architectures with Apache Pulsar and Apache ...
 
Understanding Broker Load Balancing - Pulsar Summit SF 2022
Understanding Broker Load Balancing - Pulsar Summit SF 2022Understanding Broker Load Balancing - Pulsar Summit SF 2022
Understanding Broker Load Balancing - Pulsar Summit SF 2022
 
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
Building an Asynchronous Application Framework with Python and Pulsar - Pulsa...
 
Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022
Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022
Pulsar's Journey in Yahoo!: On-prem, Cloud and Hybrid - Pulsar Summit SF 2022
 
Event-Driven Applications Done Right - Pulsar Summit SF 2022
Event-Driven Applications Done Right - Pulsar Summit SF 2022Event-Driven Applications Done Right - Pulsar Summit SF 2022
Event-Driven Applications Done Right - Pulsar Summit SF 2022
 
Pulsar @ Scale. 200M RPM and 1K instances - Pulsar Summit SF 2022
Pulsar @ Scale. 200M RPM and 1K instances - Pulsar Summit SF 2022Pulsar @ Scale. 200M RPM and 1K instances - Pulsar Summit SF 2022
Pulsar @ Scale. 200M RPM and 1K instances - Pulsar Summit SF 2022
 
Data Democracy: Journey to User-Facing Analytics - Pulsar Summit SF 2022
Data Democracy: Journey to User-Facing Analytics - Pulsar Summit SF 2022Data Democracy: Journey to User-Facing Analytics - Pulsar Summit SF 2022
Data Democracy: Journey to User-Facing Analytics - Pulsar Summit SF 2022
 
Beam + Pulsar: Powerful Stream Processing at Scale - Pulsar Summit SF 2022
Beam + Pulsar: Powerful Stream Processing at Scale - Pulsar Summit SF 2022Beam + Pulsar: Powerful Stream Processing at Scale - Pulsar Summit SF 2022
Beam + Pulsar: Powerful Stream Processing at Scale - Pulsar Summit SF 2022
 
Welcome and Opening Remarks - Pulsar Summit SF 2022
Welcome and Opening Remarks - Pulsar Summit SF 2022Welcome and Opening Remarks - Pulsar Summit SF 2022
Welcome and Opening Remarks - Pulsar Summit SF 2022
 
Log System As Backbone – How We Built the World’s Most Advanced Vector Databa...
Log System As Backbone – How We Built the World’s Most Advanced Vector Databa...Log System As Backbone – How We Built the World’s Most Advanced Vector Databa...
Log System As Backbone – How We Built the World’s Most Advanced Vector Databa...
 
Improvements Made in KoP 2.9.0 - Pulsar Summit Asia 2021
Improvements Made in KoP 2.9.0  - Pulsar Summit Asia 2021Improvements Made in KoP 2.9.0  - Pulsar Summit Asia 2021
Improvements Made in KoP 2.9.0 - Pulsar Summit Asia 2021
 
Pulsar in the Lakehouse: Overview of Apache Pulsar and Delta Lake Connector -...
Pulsar in the Lakehouse: Overview of Apache Pulsar and Delta Lake Connector -...Pulsar in the Lakehouse: Overview of Apache Pulsar and Delta Lake Connector -...
Pulsar in the Lakehouse: Overview of Apache Pulsar and Delta Lake Connector -...
 

Recently uploaded

+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
?#DUbAI#??##{{(☎️+971_581248768%)**%*]'#abortion pills for sale in dubai@
 
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers:  A Deep Dive into Serverless Spatial Data and FMECloud Frontiers:  A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
Safe Software
 

Recently uploaded (20)

Automating Google Workspace (GWS) & more with Apps Script
Automating Google Workspace (GWS) & more with Apps ScriptAutomating Google Workspace (GWS) & more with Apps Script
Automating Google Workspace (GWS) & more with Apps Script
 
Top 5 Benefits OF Using Muvi Live Paywall For Live Streams
Top 5 Benefits OF Using Muvi Live Paywall For Live StreamsTop 5 Benefits OF Using Muvi Live Paywall For Live Streams
Top 5 Benefits OF Using Muvi Live Paywall For Live Streams
 
Exploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone ProcessorsExploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone Processors
 
Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024
 
Understanding Discord NSFW Servers A Guide for Responsible Users.pdf
Understanding Discord NSFW Servers A Guide for Responsible Users.pdfUnderstanding Discord NSFW Servers A Guide for Responsible Users.pdf
Understanding Discord NSFW Servers A Guide for Responsible Users.pdf
 
Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024
 
Strategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a FresherStrategies for Landing an Oracle DBA Job as a Fresher
Strategies for Landing an Oracle DBA Job as a Fresher
 
Deploy with confidence: VMware Cloud Foundation 5.1 on next gen Dell PowerEdg...
Deploy with confidence: VMware Cloud Foundation 5.1 on next gen Dell PowerEdg...Deploy with confidence: VMware Cloud Foundation 5.1 on next gen Dell PowerEdg...
Deploy with confidence: VMware Cloud Foundation 5.1 on next gen Dell PowerEdg...
 
+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
+971581248768>> SAFE AND ORIGINAL ABORTION PILLS FOR SALE IN DUBAI AND ABUDHA...
 
Workshop - Best of Both Worlds_ Combine KG and Vector search for enhanced R...
Workshop - Best of Both Worlds_ Combine  KG and Vector search for  enhanced R...Workshop - Best of Both Worlds_ Combine  KG and Vector search for  enhanced R...
Workshop - Best of Both Worlds_ Combine KG and Vector search for enhanced R...
 
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers:  A Deep Dive into Serverless Spatial Data and FMECloud Frontiers:  A Deep Dive into Serverless Spatial Data and FME
Cloud Frontiers: A Deep Dive into Serverless Spatial Data and FME
 
Manulife - Insurer Innovation Award 2024
Manulife - Insurer Innovation Award 2024Manulife - Insurer Innovation Award 2024
Manulife - Insurer Innovation Award 2024
 
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data DiscoveryTrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
TrustArc Webinar - Unlock the Power of AI-Driven Data Discovery
 
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemkeProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
ProductAnonymous-April2024-WinProductDiscovery-MelissaKlemke
 
Data Cloud, More than a CDP by Matt Robison
Data Cloud, More than a CDP by Matt RobisonData Cloud, More than a CDP by Matt Robison
Data Cloud, More than a CDP by Matt Robison
 
Apidays New York 2024 - The value of a flexible API Management solution for O...
Apidays New York 2024 - The value of a flexible API Management solution for O...Apidays New York 2024 - The value of a flexible API Management solution for O...
Apidays New York 2024 - The value of a flexible API Management solution for O...
 
A Year of the Servo Reboot: Where Are We Now?
A Year of the Servo Reboot: Where Are We Now?A Year of the Servo Reboot: Where Are We Now?
A Year of the Servo Reboot: Where Are We Now?
 
Boost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivityBoost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivity
 
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
Bajaj Allianz Life Insurance Company - Insurer Innovation Award 2024
 
The 7 Things I Know About Cyber Security After 25 Years | April 2024
The 7 Things I Know About Cyber Security After 25 Years | April 2024The 7 Things I Know About Cyber Security After 25 Years | April 2024
The 7 Things I Know About Cyber Security After 25 Years | April 2024
 

Message Redelivery: An Unexpected Journey - Pulsar Summit SF 2022

  • 1. Pulsar Summit San Francisco Hotel Nikko August 18 2022 Tech Deep Dive Message Redelivery: An Unexpected Journey David Kjerrumgaard Developer Advocate • StreamNative
  • 2. • Apache Pulsar Committer & author of Pulsar In Action • Former Principal Software Engineer on Splunk’s Pulsar-as-a-Service platform • Over 15 years experience in Big Data/Streaming David Kjerrumgaard Developer Advocate StreamNative
  • 4. streamnative.io 10,000 Foot Level ● Pulsar Clients and Brokers interact with one another by exchanging Command Messages back-and-forth. ● Brokers are continually listening for incoming commands on a known port. ● Clients create a connection to the Broker on this port and start issuing commands.
  • 5. streamnative.io Command Categories ● These commands can be broken down into different categories based on how they are used.
  • 6. streamnative.io Client Commands ● These commands are used by the Pulsar Client & Broker to establish communication, authenticate, and perform topic lookups. ● Commands are used to transition the client through various states, e.g., disconnected to connected.
  • 7. Client Creation ● When the build() method is called, under the covers the CONNECT command is sent to the address specified by the serviceUrl property. ● If the client is authenticated, then the Pulsar Broker will respond with a CONNECTED command.
  • 8. streamnative.io Client Creation Flow ● The builder dispatches a CONNECT command (including credentials) to establish a connection. ● The Broker authenticates the client and sends a CONNECTED command to indicate success.
  • 10. streamnative.io Producer Commands ● These commands are used to move messages from the Producer to the Broker. ● Commands are used to perform message handshaking between the Producer and the Broker, e.g., SEND/SEND_RECEIPT
  • 11. Producer Creation ● When the create() method is called, under the covers a PRODUCER command is sent to the Broker. ● If the producer is authorized and has a compatible schema, then the Pulsar Broker will respond with a PRODUCER_SUCCESS command.
  • 12. Producer Creation – Configuration ProducerBuilder Method Description enableBatching (boolean enable) Control whether automatic batching of messages is enabled for the producer. batchingMaxBytes (int max) Set the maximum number of bytes permitted in a batch batchingMaxMessages (int max) Set the maximum number of messages permitted in a batch. batchingMaxPublishDelay (long delay, TimeUnit timeUnit) Set the time period within which the messages sent will be batched maxPendingMessages (int max) Set the max size of the queue holding the messages pending to receive an acknowledgment from the broker. sendTimeout(int timeout, TimeUnit unit) If a message is not acknowledged by the server before the sendTimeout expires, an error will be reported.
  • 13. streamnative.io Producer Creation Flow ● The PRODUCER command includes additional info such as the Schema, etc. ● The Broker creates a Producer object on the Broker side and sends a PRODUCER_SUCCESS command if successful.
  • 14. Broker-Side Producer Object ● An org.apache.pulsar.broker.service.Producer object is created on the broker-side that retains a reference to the corresponding Topic object. ● The Topic object is responsible for writing all the received messages to the Managed Ledger.
  • 15. Client-Side Producer Object ● A Producer object is created on the client-side that has a BatchMessageContainer object that is used to retain messages until the batch is considered full. ● Messages published via the send()/sendAsync() methods are stored in the BatchMessageContainer. ● When the batch is full, messages are removed from the BatchMessageContainer and moved to a pendingMessages queue to await acknowledgement.
  • 16.
  • 17. Message Publication ● The producing application can call the send() method multiple times. ● Each of these messages are added to the BatchMessageContainer until a batch send is triggered due to size or time.
  • 18. Message Publication – cont. ● Once the batch is considered “full”, the raw messages are prepared for transmission to the Broker. ● Each message is annotated with metadata such as the producer's name, sequenceID, and creation time.
  • 19. Message Publication – cont. ● The wrapped messages are added to the pendingMessage queue and sent to the Broker via a SEND command.
  • 20. Message Publication – Success. ● Once the message is persisted to BookKeeper the Broker notifies the Producer via a SEND_RECEIPT command and the message is removed from the pendingMessage queue.
  • 21. Publication Redelivery Scenario #1 ● If the message cannot be persisted, the Broker notifies the Producer via a SEND_ERROR command and the message will be resent from the pendingMessage queue.
  • 22. Publication Redelivery Scenario #1 – cont ● If the PRODUCER receives a SEND_ERROR message, then it will resend all the messages in the pendingMessages queue.
  • 23. Publication Redelivery Scenario #2 ● If there is an issue during the transmission of the message, then it remains in the pendingMessages queue and will be resent will the Producer reconnects.
  • 24. Publication Redelivery Scenario #2 ● After the Producer reconnects, all the messages in the pendingMessages queue are re-sent automatically.
  • 26. streamnative.io Consumer Commands ● These commands are used to move messages from the Broker to the Consumer. ● Commands are used to perform message handshaking between the Consumer and the Broker, e.g., MESSAGE/ACK
  • 27. Consumer Creation ● When the subscribe() method is called, under the covers a SUBSCRIBE command is sent to the Broker. ● If the Consumer is authorized and has specified a compatible schema, then the Pulsar Broker will respond with a SUCCESS command.
  • 28. Consumer Creation – Configuration ConsumerBuilder Method Description acknowledgmentGroupTime (long delay, TimeUnit unit) Group the consumer acknowledgments for the specified time. ackTimeout(long ackTimeout, TimeUnit timeUnit) Set the timeout for unacked messages ackTimeoutTickTime (long tickTime, TimeUnit timeUnit) Define the granularity of the ack-timeout redelivery. ackTimeoutRedeliveryBackoff (Redeli veryBackoff backoff) Allows you to specify a custom redelivery strategy for message that have exceeded the ack timeout negativeAckRedeliveryDelay (long delay, TimeUnit timeUnit) Set the delay to wait before re-delivering messages that have failed to be process. negativeAckRedeliveryBackoff (Redel iveryBackoff backoff) Allows you to specify a custom redelivery strategy for negatively acknowledged messages
  • 29. streamnative.io Consumer Creation Flow ● The SUBSCRIBE command includes additional info such as the Schema, etc. ● The Broker creates a Consumer object on the Broker side and sends a SUCCESS command if successful.
  • 33. Message Flow ● A pull-based model controlled by the FLOW command.
  • 34. Message Flow - Backend ● The messages are chosen by the Subscription and read from BookKeeper.
  • 35. Message Flow – cont. ● The messages are added to the pendingAcks HashMap before being sent back via individual MESSAGE commands.
  • 36. Message Consumption ● When you consume messages using the receive() / receiveAsync() methods, the messages are taken from the incomingMessages queue and given to the application.
  • 37. Automated Message Flow ● The client-side Consumer periodically requests new messages from the broker-side Consumer (pull-based) ● This process is triggered automatically when the client-side Consumer’s incomingMessages queue drops below 50% of its capacity. ● When this condition is detected a FLOW command is dispatched to the broker-side Consumer to request another batch of messages.
  • 38. Post-Processing Options ● After the message is processed, it can either be acknowledged, negatively acknowledged, or sent to the retry letter topic to be re-delivered after a specified length of time.
  • 40. Message Acknowledgement ● After a message has been successfully consumed, it must be acknowledged to prevent redelivery/reprocessing. ● The behavior of the Acknowledgment process can be controlled by the following settings in the ConsumerBuilder • acknowledgmentGroupTime(long delay, TimeUnit unit) • ackTimeout(long ackTimeout, TimeUnit timeUnit) • ackTimeoutRedeliveryBackoff(RedeliveryBackoff ackBackoff) • ackTimeoutTickTime(long tickTime, TimeUnit timeUnit)
  • 41. Message Acknowledgement ● If the acknowledgmentGroupTime property is set, then the acks will be grouped together for the specified interval before being sent together, otherwise an ack is sent immediately.
  • 42. Message Acknowledgement - Broker ● On the Broker side, the ack is removed from the pendingAcks list, and the subscription updates the cursor position. An ACK_RESPONSE command is sent back to the client to complete the handshake.
  • 43. Redelivery of Messages Due to Ack Timeout
  • 44. Waiting ● On the client-side, the UnackedMessageTracker tracks the receipt of the ACK commands that were sent to the Broker to ensure that the subscription cursor is positioned correctly. ● Any ACKs that do not receive a corresponding ACK_RESPONSE command must be resent to complete the handshake. Missing ACK_RESPONSEs indicate the possibility of a network outage, etc. ● How long the client decided to wait for these ACK_RESPONSEs is referred to as the “Ack Timeout”
  • 45. The UnackedMessageTracker ● Messages are added to the UnackedMessageTracker as their corresponding ACKs are sent to the Broker.
  • 46. The UnackedMessageTracker ● To ensure that we only redeliver messages that have not been ACKed by the Broker, we remove message as their corresponding ACK_RESPONSEs are received from the Broker.
  • 47. The UnackedMessageTracker ● There are 3 variations of the UnackedMessageTracker, and the one that is used is determined by the way that the consumer was configured.
  • 48. Ack Timeout Configuration ● If the ackTimeout property is set to zero, then ACK tracking is disabled, and we have AT_MOST_ONCE processing semantics. ● Otherwise, we use one of the two timer-based variations that waits a period for the ACK_RESPONSEs before resending the ACKs to the Broker. ● The only difference between the two is how the ack timeout interval is calculated.
  • 49. Timer-Based UnackedMessgeTrackers ● Internally, both timer-based UnackedMessageTracker versions use a Timer task to trigger the redelivery of the ACKs. ● The base UnackedMessageTracker operates on a fixed interval equal to either the ackTimeoutTickTime property (if set) or else the ackTimeout property. ● The UnackedMessageRedeliveryTracker operates on a variable interval based upon the value of the ackTimeoutRedeliveryBackoff property.
  • 50. Timer-Based UnackedMessgeTrackers ● Here is an example of an UnackedMessageRedeliveryTracker configured to use an Exponential backoff strategy.
  • 51. Ack Timeout ● Once the ack timeout interval has elapsed, the consumer sends a REDELIVER_UNACKNOWLEDGED_MESSAGES command to the Broker.
  • 52. Ack Timeout ● The REDELIVER_UNACKNOWLEDGED_MESSAGES triggers the Broker-side consumer to resend ALL the messages in PendingAcks.
  • 54. The NegativeAcksTracker ● Messages are added to the NegativeAcksTracker when the application calls consumer.negativeAck(msg)
  • 55. Negative Acks Configuration ● There is only one variation of the NegativeAcksTracker, and it uses a Timer task to trigger the redelivery of the negatively acked messages. ● The Timer operates on a fixed interval if the negativeAckRedeliveryDelay property is set. Otherwise, it uses the strategy specified by the negativeAckRedeliveryBackoff property.
  • 56. Timeout ● Once the redelivery delay has elapsed, the consumer sends a REDELIVER_UNACKNOWLEDGED_MESSAGES command to the Broker.
  • 58. Delayed Message Delivery ● If the enableRetry property is set to true, the consumer can request that a message be re-delivered after a specified delay, e.g., consumer.reconsumeLater(msg, 3, TimeUnit.SECONDS) ● Under the covers, this process utilizes the deliverAfter(delayTime, unit) method of the Producer class.
  • 59. Delayed Message Delivery ● When you first call the reconsumeLater method on the consumer, a Producer object is initialized using the following code block.
  • 60. Delayed Message Delivery ● Next, we check to see how many times we have already tried to redeliver this message by examining the SYSTEM_PROPERTY_RECONSUMETIMES property in the Message’s properties. ● If that value exceeds the maxRedeliverCount setting of the DLQ policy, then the message is routed to the DLQ. ● Otherwise, it is scheduled for delayed deliver via a call to producer.newMessage().deliverAfter(delayTime, unit);
  • 61. streamnative.io Key Takeaways ➔ Message redelivery can occur when Producing or Consuming messages in Pulsar ➔ When producing, messages get re-delivered if the Broker doesn’t acknowledge receipt of the message in a timely manner. ➔ When consuming, messages get re-delivered under 3 different circumstances. Ack timeout, negative ack, and delay
  • 62. David Kjerrumgaard Thank you! david@streamnative.io @DavidKjerrumga1 Pulsar Summit San Francisco Hotel Nikko August 18 2022