Messaging
Pulsar is built on the publish-subscribe pattern (often abbreviated to pub-sub). In this pattern, producers publish messages to topics; consumers subscribe to those topics, process incoming messages, and send acknowledgments to the broker when processing is finished.
When a subscription is created, Pulsar retains all messages, even if the consumer is disconnected. The retained messages are discarded only when a consumer acknowledges that all these messages are processed successfully.
If the consumption of a message fails and you want this message to be consumed again, you can enable the message redelivery mechanism to request the broker to resend this message.
Messages
Messages are the basic “unit” of Pulsar. They’re what producers publish to topics and what consumers then consume from topics. The following table lists the components of messages.
Component | Description |
---|---|
Value / data payload | The data carried by the message. All Pulsar messages contain raw bytes, although message data can also conform to data schemas. |
Key | The key (string type) of the message. It is a short name of message key or partition key. Messages are optionally tagged with keys, which is useful for features like topic compaction. |
Properties | An optional key/value map of user-defined properties. |
Producer name | The name of the producer who produces the message. If you do not specify a producer name, the default name is used. |
Topic name | The name of the topic that the message is published to. |
Schema version | The version number of the schema that the message is produced with. |
Sequence ID | Each Pulsar message belongs to an ordered sequence on its topic. The sequence ID of a message is initially assigned by its producer, indicating its order in that sequence, and can also be customized. Sequence ID can be used for message deduplication. If brokerDeduplicationEnabled is set to true , the sequence ID of each message is unique within a producer of a topic (non-partitioned) or a partition. |
Message ID | The message ID of a message is assigned by bookies as soon as the message is persistently stored. Message ID indicates a message’s specific position in a ledger and is unique within a Pulsar cluster. |
Publish time | The timestamp of when the message is published. The timestamp is automatically applied by the producer. |
Event time | An optional timestamp attached to a message by applications. For example, applications attach a timestamp on when the message is processed. If nothing is set to event time, the value is 0 . |
The default max size of a message is 5 MB. You can configure the max size of a message with the following configuration options.
In the
broker.conf
file.# The max size of a message (in bytes).
maxMessageSize=5242880
In the
bookkeeper.conf
file.# The max size of the netty frame (in bytes). Any messages received larger than this value are rejected. The default value is 5 MB.
nettyMaxFrameSizeBytes=5253120
For more information on Pulsar messages, see Pulsar binary protocol.
Acknowledgment
A message acknowledgment is sent by a consumer to a broker after the consumer consumes a message successfully. Then, this consumed message will be permanently stored and deleted only after all the subscriptions have acknowledged it. An acknowledgment (ack) is Pulsar’s way of knowing that the message can be deleted from the system. If you want to store the messages that have been acknowledged by a consumer, you need to configure the message retention policy.
For batch messages, you can enable batch index acknowledgment to avoid dispatching acknowledged messages to the consumer. For details about batch index acknowledgment, see batching.
Messages can be acknowledged in one of the following two ways:
Being acknowledged individually
With individual acknowledgment, the consumer acknowledges each message and sends an acknowledgment request to the broker.
Being acknowledged cumulatively
With cumulative acknowledgment, the consumer only acknowledges the last message it received. All messages in the stream up to (and including) the provided message are not redelivered to that consumer.
If you want to acknowledge messages individually, you can use the following API.
consumer.acknowledge(msg);
If you want to acknowledge messages cumulatively, you can use the following API.
consumer.acknowledgeCumulative(msg);
note
Cumulative acknowledgment cannot be used in Shared or Key_shared subscription type, because Shared or Key_Shared subscription type involves multiple consumers which have access to the same subscription. In Shared and Key_Shared subscription types, messages should be acknowledged individually.
Negative acknowledgment
The negative acknowledgment mechanism allows you to send a notification to the broker indicating the consumer did not process a message. When a consumer fails to consume a message and needs to re-consume it, the consumer sends a negative acknowledgment (nack) to the broker, triggering the broker to redeliver this message to the consumer.
Messages are negatively acknowledged individually or cumulatively, depending on the consumption subscription type.
In Exclusive and Failover subscription types, consumers only negatively acknowledge the last message they receive.
In Shared and Key_Shared subscription types, consumers can negatively acknowledge messages individually.
Be aware that negative acknowledgments on ordered subscription types, such as Exclusive, Failover and Key_Shared, might cause failed messages being sent to consumers out of the original order.
If you are going to use negative acknowledgment on a message, make sure it is negatively acknowledged before the acknowledgment timeout.
Use the following API to negatively acknowledge message consumption.
Consumer<byte[]> consumer = pulsarClient.newConsumer()
.topic(topic)
.subscriptionName("sub-negative-ack")
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
.negativeAckRedeliveryDelay(2, TimeUnit.SECONDS) // the default value is 1 min
.subscribe();
Message<byte[]> message = consumer.receive();
// call the API to send negative acknowledgment
consumer.negativeAcknowledge(message);
message = consumer.receive();
consumer.acknowledge(message);
To redeliver messages with different delays, you can use the redelivery backoff mechanism by setting the number of retries to deliver the messages. Use the following API to enable Negative Redelivery Backoff
.
Consumer<byte[]> consumer = pulsarClient.newConsumer()
.topic(topic)
.subscriptionName("sub-negative-ack")
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
.negativeAckRedeliveryBackoff(MultiplierRedeliveryBackoff.builder()
.minDelayMs(1000)
.maxDelayMs(60 * 1000)
.multiplier(2)
.build())
.subscribe();
The message redelivery behavior should be as follows.
Redelivery count | Redelivery delay |
---|---|
1 | 1 seconds |
2 | 2 seconds |
3 | 4 seconds |
4 | 8 seconds |
5 | 16 seconds |
6 | 32 seconds |
7 | 60 seconds |
8 | 60 seconds |
note
If batching is enabled, all messages in one batch are redelivered to the consumer.
Acknowledgment timeout
note
By default, the acknowledge timeout is disabled and that means that messages delivered to a consumer will not be re-delivered unless the consumer crashes.
The acknowledgment timeout mechanism allows you to set a time range during which the client tracks the unacknowledged messages. After this acknowledgment timeout (ackTimeout
) period, the client sends redeliver unacknowledged messages
request to the broker, thus the broker resends the unacknowledged messages to the consumer.
You can configure the acknowledgment timeout mechanism to redeliver the message if it is not acknowledged after ackTimeout
or to execute a timer task to check the acknowledgment timeout messages during every ackTimeoutTickTime
period.
You can also use the redelivery backoff mechanism to redeliver messages with different delays by setting the number of times the messages are retried.
If you want to use redelivery backoff, you can use the following API.
consumer.ackTimeout(10, TimeUnit.SECOND)
.ackTimeoutRedeliveryBackoff(MultiplierRedeliveryBackoff.builder()
.minDelayMs(1000)
.maxDelayMs(60 * 1000)
.multiplier(2)
.build());
The message redelivery behavior should be as follows.
Redelivery count | Redelivery delay |
---|---|
1 | 10 + 1 seconds |
2 | 10 + 2 seconds |
3 | 10 + 4 seconds |
4 | 10 + 8 seconds |
5 | 10 + 16 seconds |
6 | 10 + 32 seconds |
7 | 10 + 60 seconds |
8 | 10 + 60 seconds |
note
- If batching is enabled, all messages in one batch are redelivered to the consumer.
- Compared with acknowledgment timeout, negative acknowledgment is preferred. First, it is difficult to set a timeout value. Second, a broker resends messages when the message processing time exceeds the acknowledgment timeout, but these messages might not need to be re-consumed.
Use the following API to enable acknowledgment timeout.
Consumer<byte[]> consumer = pulsarClient.newConsumer()
.topic(topic)
.ackTimeout(2, TimeUnit.SECONDS) // the default value is 0
.ackTimeoutTickTime(1, TimeUnit.SECONDS)
.subscriptionName("sub")
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
.subscribe();
Message<byte[]> message = consumer.receive();
// wait at least 2 seconds
message = consumer.receive();
consumer.acknowledge(message);
Retry letter topic
Retry letter topic allows you to store the messages that failed to be consumed and retry consuming them later. With this method, you can customize the interval at which the messages are redelivered. Consumers on the original topic are automatically subscribed to the retry letter topic as well. Once the maximum number of retries has been reached, the unconsumed messages are moved to a dead letter topic for manual processing. The functionality of a retry letter topic is implemented by consumers.
The diagram below illustrates the concept of the retry letter topic.
The intention of using retry letter topic is different from using delayed message delivery, even though both are aiming to consume a message later. Retry letter topic serves failure handling through message redelivery to ensure critical data is not lost, while delayed message delivery is intended to deliver a message with a specified time delay.
By default, automatic retry is disabled. You can set enableRetry
to true
to enable automatic retry on the consumer.
Use the following API to consume messages from a retry letter topic. When the value of maxRedeliverCount
is reached, the unconsumed messages are moved to a dead letter topic.
Consumer<byte[]> consumer = pulsarClient.newConsumer(Schema.BYTES)
.topic("my-topic")
.subscriptionName("my-subscription")
.subscriptionType(SubscriptionType.Shared)
.enableRetry(true)
.deadLetterPolicy(DeadLetterPolicy.builder()
.maxRedeliverCount(maxRedeliveryCount)
.build())
.subscribe();
The default retry letter topic uses this format:
<topicname>-<subscriptionname>-RETRY
note
- For Pulsar 2.6.x and 2.7.x, the default retry letter topic uses the format of
<subscriptionname>-RETRY
. If you upgrade from 2.6.x~2.7.x to 2.8.x or later, you need to delete historical retry letter topics and retry letter partitioned topics. Otherwise, Pulsar continues to use original topics, which are formatted with<subscriptionname>-RETRY
. - It is not recommended to use
<subscriptionname>-RETRY
because if multiple topics under the same namespace have the same subscription, then retry message topic names for multiple topics might be the same, which will result in mutual consumptions.
Use the Java client to specify the name of the retry letter topic.
Consumer<byte[]> consumer = pulsarClient.newConsumer(Schema.BYTES)
.topic("my-topic")
.subscriptionName("my-subscription")
.subscriptionType(SubscriptionType.Shared)
.enableRetry(true)
.deadLetterPolicy(DeadLetterPolicy.builder()
.maxRedeliverCount(maxRedeliveryCount)
.retryLetterTopic("my-retry-letter-topic-name")
.build())
.subscribe();
The messages in the retry letter topic contain some special properties that are automatically created by the client.
Special property | Description |
---|---|
REAL_TOPIC | The real topic name. |
ORIGIN_MESSAGE_ID | The origin message ID. It is crucial for message tracking. |
RECONSUMETIMES | The number of retries to consume messages. |
DELAY_TIME | Message retry interval in milliseconds. |
Example
REAL_TOPIC = persistent://public/default/my-topic
ORIGIN_MESSAGE_ID = 1:0:-1:0
RECONSUMETIMES = 6
DELAY_TIME = 3000
Use the following API to store the messages in a retrial queue.
consumer.reconsumeLater(msg, 3, TimeUnit.SECONDS);
Use the following API to add custom properties for the reconsumeLater
function. In the next attempt to consume, custom properties can be get from message#getProperty.
Map<String, String> customProperties = new HashMap<String, String>();
customProperties.put("custom-key-1", "custom-value-1");
customProperties.put("custom-key-2", "custom-value-2");
consumer.reconsumeLater(msg, customProperties, 3, TimeUnit.SECONDS);
note
- Currently, retry letter topic is enabled in Shared subscription types.
- Compared with negative acknowledgment, retry letter topic is more suitable for messages that require a large number of retries with a configurable retry interval. Because messages in the retry letter topic are persisted to BookKeeper, while messages that need to be retried due to negative acknowledgment are cached on the client side.
Dead letter topic
Dead letter topic allows you to continue message consumption even when some messages are not consumed successfully. The messages that have failed to be consumed are stored in a specific topic, which is called the dead letter topic. The functionality of a dead letter topic is implemented by consumers. You can decide how to handle the messages in the dead letter topic.
Enable dead letter topic in a Java client using the default dead letter topic.
Consumer<byte[]> consumer = pulsarClient.newConsumer(Schema.BYTES)
.topic("my-topic")
.subscriptionName("my-subscription")
.subscriptionType(SubscriptionType.Shared)
.deadLetterPolicy(DeadLetterPolicy.builder()
.maxRedeliverCount(maxRedeliveryCount)
.build())
.subscribe();
The default dead letter topic uses this format:
<topicname>-<subscriptionname>-DLQ
The dead letter producerName uses this format:
<topicname>-<subscriptionname>-<consumername>-DLQ
note
- For Pulsar 2.6.x and 2.7.x, the default dead letter topic uses the format of
<subscriptionname>-DLQ
. If you upgrade from 2.6.x~2.7.x to 2.8.x or later, you need to delete historical dead letter topics and retry letter partitioned topics. Otherwise, Pulsar continues to use original topics, which are formatted with<subscriptionname>-DLQ
. - It is not recommended to use
<subscriptionname>-DLQ
because if multiple topics under the same namespace have the same subscription, then dead message topic names for multiple topics might be the same, which will result in mutual consumptions.
Use the Java client to specify the name of the dead letter topic.
Consumer<byte[]> consumer = pulsarClient.newConsumer(Schema.BYTES)
.topic("my-topic")
.subscriptionName("my-subscription")
.subscriptionType(SubscriptionType.Shared)
.deadLetterPolicy(DeadLetterPolicy.builder()
.maxRedeliverCount(maxRedeliveryCount)
.deadLetterTopic("my-dead-letter-topic-name")
.build())
.subscribe();
By default, there is no subscription during DLQ topic creation. Without a just-in-time subscription to the DLQ topic, you may lose messages. To automatically create an initial subscription for the DLQ, you can specify the initialSubscriptionName
parameter. If this parameter is set but the broker’s allowAutoSubscriptionCreation
is disabled, the DLQ producer will fail to be created.
Consumer<byte[]> consumer = pulsarClient.newConsumer(Schema.BYTES)
.topic("my-topic")
.subscriptionName("my-subscription")
.subscriptionType(SubscriptionType.Shared)
.deadLetterPolicy(DeadLetterPolicy.builder()
.maxRedeliverCount(maxRedeliveryCount)
.deadLetterTopic("my-dead-letter-topic-name")
.initialSubscriptionName("init-sub")
.build())
.subscribe();
Dead letter topic serves message redelivery, which is triggered by acknowledgment timeout or negative acknowledgment or retry letter topic.
note
Currently, dead letter topic is enabled in Shared and Key_Shared subscription types.
Compression
Message compression can reduce message size by paying some CPU overhead. The Pulsar client supports the following compression types:
Compression types are stored in the message metadata, so consumers can adopt different compression types automatically, as needed.
The sample code below shows how to enable compression type for a producer:
client.newProducer()
.topic("topic-name")
.compressionType(CompressionType.LZ4)
.create();
Batching
When batching is enabled, the producer accumulates and sends a batch of messages in a single request. The batch size is defined by the maximum number of messages and the maximum publish latency. Therefore, the backlog size represents the total number of batches instead of the total number of messages.
In Pulsar, batches are tracked and stored as single units rather than as individual messages. Consumers unbundle a batch into individual messages. However, scheduled messages (configured through the deliverAt
or the deliverAfter
parameter) are always sent as individual messages even when batching is enabled.
In general, a batch is acknowledged when all of its messages are acknowledged by a consumer. It means that when not all batch messages are acknowledged, then unexpected failures, negative acknowledgments, or acknowledgment timeouts can result in a redelivery of all messages in this batch.
To avoid redelivering acknowledged messages in a batch to the consumer, Pulsar introduces batch index acknowledgment since Pulsar 2.6.0. When batch index acknowledgment is enabled, the consumer filters out the batch index that has been acknowledged and sends the batch index acknowledgment request to the broker. The broker maintains the batch index acknowledgment status and tracks the acknowledgment status of each batch index to avoid dispatching acknowledged messages to the consumer. The batch is deleted when all indices of the messages in it are acknowledged.
By default, batch index acknowledgment is disabled (acknowledgmentAtBatchIndexLevelEnabled=false
). You can enable batch index acknowledgment by setting the acknowledgmentAtBatchIndexLevelEnabled
parameter to true
at the broker side. Enabling batch index acknowledgment results in more memory overheads.
Batch index acknowledgment must also be enabled in the consumer by calling .enableBatchIndexAcknowledgment(true);
For example:
Consumer<byte[]> consumer = pulsarClient.newConsumer()
.topic(topicName)
.subscriptionName(subscriptionName)
.subscriptionType(subType)
.enableBatchIndexAcknowledgment(true)
.subscribe();
note
When using the synchronous send
method for producing messages, the batch will be sent immediately even if it is not full. This helps reduce message sending latency and prevents blocking of the caller’s thread. When producing messages in a single thread, you should use the asynchronous sendAsync
method to send messages in batches.
Chunking
Message chunking enables Pulsar to process large payload messages by splitting the message into chunks at the producer side and aggregating chunked messages at the consumer side.
With message chunking enabled, when the size of a message exceeds the allowed maximum payload size (the maxMessageSize
parameter of broker), the workflow of messaging is as follows:
- The producer splits the original message into chunked messages and publishes them with chunked metadata to the broker separately and in order.
- The broker stores the chunked messages in one managed ledger in the same way as that of ordinary messages, and it uses the
chunkedMessageRate
parameter to record chunked message rate on the topic. - The consumer buffers the chunked messages and aggregates them into the receiver queue when it receives all the chunks of a message.
- The client consumes the aggregated message from the receiver queue.
note
- Chunking is only available for persistent topics.
- Chunking cannot be enabled simultaneously with batching. Before enabling chunking, you need to disable batching.
Handle consecutive chunked messages with one ordered consumer
The following figure shows a topic with one producer that publishes a large message payload in chunked messages along with regular non-chunked messages. The producer publishes message M1 in three chunks labeled M1-C1, M1-C2 and M1-C3. The broker stores all the three chunked messages in the managed ledger and dispatches them to the ordered (exclusive/failover) consumer in the same order. The consumer buffers all the chunked messages in memory until it receives all the chunked messages, aggregates them into one message and then hands over the original message M1 to the client.
Handle interwoven chunked messages with one ordered consumer
When multiple producers publish chunked messages into a single topic, the broker stores all the chunked messages coming from different producers in the same managed ledger. The chunked messages in the managed ledger can be interwoven with each other. As shown below, Producer 1 publishes message M1 in three chunks M1-C1, M1-C2 and M1-C3. Producer 2 publishes message M2 in three chunks M2-C1, M2-C2 and M2-C3. All chunked messages of the specific message are still in order but might not be consecutive in the managed ledger.
note
In this case, interwoven chunked messages may bring some memory pressure to the consumer because the consumer keeps a separate buffer for each large message to aggregate all its chunks in one message. You can limit the maximum number of chunked messages a consumer maintains concurrently by configuring the maxPendingChunkedMessage
parameter. When the threshold is reached, the consumer drops pending messages by silently acknowledging them or asking the broker to redeliver them later, optimizing memory utilization.
Enable Message Chunking
Prerequisite: Disable batching by setting the enableBatching
parameter to false
.
The message chunking feature is OFF by default. To enable message chunking, set the chunkingEnabled
parameter to true
when creating a producer.
note
If the consumer fails to receive all chunks of a message within a specified period, it expires incomplete chunks. The default value is 1 minute. For more information about the expireTimeOfIncompleteChunkedMessage
parameter, refer to org.apache.pulsar.client.api.
Topics
A Pulsar topic is a unit of storage that organizes messages into a stream. As in other pub-sub systems, topics in Pulsar are named channels for transmitting messages from producers to consumers. Topic names are URLs that have a well-defined structure:
{persistent|non-persistent}://tenant/namespace/topic
Topic name component | Description |
---|---|
persistent / non-persistent | This identifies the type of topic. Pulsar supports two kind of topics: persistent and non-persistent. The default is persistent, so if you do not specify a type, the topic is persistent. With persistent topics, all messages are durably persisted on disks (if the broker is not standalone, messages are durably persisted on multiple disks), whereas data for non-persistent topics is not persisted to storage disks. |
tenant | The topic tenant within the instance. Tenants are essential to multi-tenancy in Pulsar, and spread across clusters. |
namespace | The administrative unit of the topic, which acts as a grouping mechanism for related topics. Most topic configuration is performed at the namespace level. Each tenant has one or more namespaces. |
topic | The final part of the name. Topic names have no special meaning in a Pulsar instance. |
note
You do not need to explicitly create topics in Pulsar. If a client attempts to write or receive messages to/from a topic that does not yet exist, Pulsar creates that topic under the namespace provided in the topic name automatically. If no tenant or namespace is specified when a client creates a topic, the topic is created in the default tenant and namespace. You can also create a topic in a specified tenant and namespace, such as persistent://my-tenant/my-namespace/my-topic
. persistent://my-tenant/my-namespace/my-topic
means the my-topic
topic is created in the my-namespace
namespace of the my-tenant
tenant.
Namespaces
A Pulsar namespace is a logical grouping of topics as well as a logical nomenclature within a tenant. A tenant creates namespaces via the admin API. For instance, a tenant with different applications can create a separate namespace for each application. A namespace allows the application to create and manage a hierarchy of topics. The topic my-tenant/app1
is a namespace for the application app1
for my-tenant
. You can create any number of topics under the namespace.
Subscriptions
A Pulsar subscription is a named configuration rule that determines how messages are delivered to consumers. It is a lease on a topic established by a group of consumers. There are four subscription types in Pulsar:
These types are illustrated in the figure below.
tip
Pub-Sub or Queuing In Pulsar, you can use different subscriptions flexibly.
- If you want to achieve traditional “fan-out pub-sub messaging” among consumers, specify a unique subscription name for each consumer. It is an exclusive subscription type.
- If you want to achieve “message queuing” among consumers, share the same subscription name among multiple consumers (shared, failover, key_shared).
- If you want to achieve both effects simultaneously, combine exclusive subscription types with other subscription types for consumers.
Subscription types
When a subscription has no consumers, its subscription type is undefined. The type of a subscription is defined when a consumer connects to it, and the type can be changed by restarting all consumers with a different configuration.
Exclusive
The exclusive type is a subscription type that only allows a single consumer to attach to the subscription. If multiple consumers subscribe to a topic using the same subscription, an error occurs. Note that if the topic is partitioned, all partitions will be consumed by the single consumer allowed to be connected to the subscription.
In the diagram below, only Consumer A is allowed to consume messages.
tip
Exclusive is the default subscription type.
Failover
The failover type is a subscription type that multiple consumers can attach to the same subscription.
A master consumer is picked for a non-partitioned topic or each partition of a partitioned topic and receives messages.
When the master consumer disconnects, all (non-acknowledged and subsequent) messages are delivered to the next consumer in line.
note
In some cases, a partition may have an older active consumer processing messages while a newly switched over active consumer starts receiving new messages. This may lead to message duplication or out-of-order.
Failover | Partitioned topics
For partitioned topics, the broker sorts consumers by priority and lexicographical order of consumer name.
The broker tries to evenly assign partitions to consumers with the highest priority.
A consumer is selected by running a module operation mod (partition index, consumer index)
.
If the number of partitions in a partitioned topic is less than the number of consumers:
For example, in the diagram below, this partitioned topic has 2 partitions and there are 4 consumers.
Each partition has 1 active consumer and 3 stand-by consumers.
For P0, Consumer A is the master consumer, while Consumer B, Consumer C, and Consumer D would be the next consumer in line to receive messages if consumer A is disconnected.
For P1, Consumer B is the master consumer, while Consumer A, Consumer C, and Consumer D would be the next consumer in line to receive messages if consumer B is disconnected.
Moreover, if Consumer A and consumer B are disconnected, then
for P0: Consumer C is the active consumer and Consumer D is the stand-by consumer.
for P1: Consumer D is the active consumer and Consumer C is the stand-by consumer.
If the number of partitions in a partitioned topic is greater than the number of consumers:
For example, in the diagram below, this partitioned topic has 9 partitions and 3 consumers.
P0, P3, and P6 are assigned to Consumer A. Consumer A is their active consumer. Consumer B and Consumer C are their stand-by consumers.
P1, P4, and P7 are assigned to Consumer B. Consumer B is their active consumer. Consumer A and Consumer C are their stand-by consumers.
P2, P5, and P8 are assigned to Consumer C. Consumer C is their active consumer. Consumer A and Consumer B are their stand-by consumers.
Failover | Non-partitioned topics
If there is one non-partitioned topic. The broker picks consumers in the order they subscribe to non-partitioned topics.
For example, in the diagram below, this non-partitioned topic has 1 topic and there are 2 consumers.
The topic has 1 active consumer and 1 stand-by consumer.
Consumer A is the master consumer, while consumer B would be the next consumer in line to receive messages if consumer A is disconnected.
If there are multiple non-partitioned topics, a consumer is selected based on consumer name hash and topic name hash. The client uses the same consumer name to subscribe to all the topics.
For example, in the diagram below, there are 4 non-partitioned topics and 2 consumers.
The non-partitioned topic 1 and non-partitioned topic 4 are assigned to consumer B. Consumer A is their stand-by consumer.
The non-partitioned topic 2 and non-partitioned topic 3 are assigned to consumer A. Consumer B is their stand-by consumer.
Shared
The shared subscription type in Pulsar allows multiple consumers to attach to the same subscription. Messages are delivered in a round-robin distribution across consumers, and any given message is delivered to only one consumer. When a consumer disconnects, all the messages that were sent to it and not acknowledged will be rescheduled for sending to the remaining consumers.
In the diagram below, Consumer A, Consumer B and Consumer C are all able to subscribe to the topic.
note
Shared subscriptions do not guarantee message ordering or support cumulative acknowledgment.
Key_Shared
The Key_Shared subscription type in Pulsar allows multiple consumers to attach to the same subscription. But different with the Shared type, messages in the Key_Shared type are delivered in distribution across consumers and messages with the same key or same ordering key are delivered to only one consumer. No matter how many times the message is re-delivered, it is delivered to the same consumer.
note
If there is a newly switched over active consumer, it will start reading messages from the position where messages are acked by the old inactive consumer.
For example, if P0 is assigned to Consumer A. Consumer A is the active consumer and Consumer B is the stand-by consumer.
If Consumer A gets disconnected without reading any messages from P0, when Consumer C is added and becomes the new active consumer, then Consumer C will start reading messages directly from P0.
If Consumer A gets disconnected after reading messages (0,1,2,3) from P0, when Consumer C is added and becomes the active consumer, then Consumer C will start reading messages (4,5,6,7) from P0.
There are three types of mapping algorithms dictating how to select a consumer for a given message key (or ordering key):
- Auto-split Hash Range
- Auto-split Consistent Hashing
- Sticky
The steps for all mapping algorithms are:
- The message key (or ordering key) is passed to a hash function (e.g., Murmur3 32-bit), yielding a 32-bit integer hash.
- That hash number is fed to the algorithm to select a consumer from the existing connected consumers.
+--------------+ +-----------+
Message Key -----> / Hash Function / ----- hash (32-bit) -------> / Algorithm / ----> Consumer
+---------------+ +----------+
When a new consumer is connected and thus added to the list of connected consumers, the algorithm re-adjusts the mapping such that some keys currently mapped to existing consumers will be mapped to the newly added consumer. When a consumer is disconnected, thus removed from the list of connected consumers, keys mapped to it will be mapped to other consumers. The sections below will explain how a consumer is selected given the message hash and how the mapping is adjusted given a new consumer is connected or an existing consumer disconnects for each algorithm.
Auto-split Hash Range
Auto-split Hash Range assumes each consumer is mapped into a single region in a range of numbers between 0 to 2^16 (65,536). So all mapped regions cover the entire range, and no regions overlap. A consumer is selected for a given key by running a modulo operation on the message hash by the range size (65,536). The number received ( 0 <= i < 65,536) is contained within a single region. The consumer mapped to that region is the one selected.
Example:
Suppose we have 4 consumers (C1, C2, C3 and C4), then:
0 16,384 32,768 49,152 65,536
|------- C3 ------|------- C2 ------|------- C1 ------|------- C4 ------|
Given a message key Order-3459134
, its hash would be murmur32("Order-3459134") = 3112179635
, and its index in the range would be 3112179635 mod 65536 = 6067
. That index is contained within region [0, 16384)
thus consumer C3 will be mapped to this message key.
When a new consumer is connected, the largest region is chosen and is then split in half - the lower half will be mapped to the newly added consumer and upper half will be mapped to the consumer owning that region. Here is how it looks like from 1 to 4 consumers:
C1 connected:
|---------------------------------- C1 ---------------------------------|
C2 connected:
|--------------- C2 ----------------|---------------- C1 ---------------|
C3 connected:
|------- C3 ------|------- C2 ------|---------------- C1 ---------------|
C4 connected:
|------- C3 ------|------- C2 ------|------- C4 ------|------- C1 ------|
When a consumer is disconnected its region will be merged into the region on its right. Examples:
C4 is disconnected:
|------- C3 ------|------- C2 ------|---------------- C1 ---------------|
C1 is disconnected:
|------- C3 ------|-------------------------- C2 -----------------------|
The advantages of this algorithm is that it affects only a single existing consumer upon add/delete consumer, at the expense of regions not evenly sized. This means some consumers gets more keys that others. The next algorithm does the other way around.
Auto-split Consistent Hashing
Auto-split Consistent Hashing assumes each consumer is mapped into a Hash Ring. It’s a range of number from 0 to 65,535 in which if you traverse the range, when reaching 65,535, the next number would be zero. It is as if you took a line starting from 0 ending at 65,535 and bent into a circle such that the end glues to the start:
65,535 ------++--------- 0
||
, - ~ ~ ~ - ,
, ' ' ,
, ,
, ,
, ,
, ,
, ,
, ,
, ,
, , '
' - , _ _ _ , '
When adding a consumer, we mark 100 points on that circle and associate them to the newly added consumer. For each number between 1 and 100, we concatenate the consumer name to that number and run the hash function on it to get the location of the point on the circle that will be marked. For Example, if the consumer name is “orders-aggregator-pod-2345-consumer” then we would mark 100 points on that circle:
murmur32("orders-aggregator-pod-2345-consumer␀0␀1") = 1003084738 % 65535 = 6028
murmur32("orders-aggregator-pod-2345-consumer␀0␀2") = 373317202 % 65535 = 29842
...
murmur32("orders-aggregator-pod-2345-consumer␀0␀100") = 320276078 % 65535 = 6533
Since the hash function has the uniform distribution attribute, those points would be uniformly distributed across the circle in random order.
C1-33
, - ~ ~ ~ - , C1-3
, ' ' ,
, ,
, , C1-45
, ,
, ,
, ,
, , C1-23
, ,
, , '
' - , _ _ _ , ' ...
A consumer is selected for a given message key by putting its hash on the circle then continue clock-wise on the circle until you reach a marking point. The point might have more than one consumer on it (hash function might have collisions). In the case of collisions, the first added consumer will handle the hash range. When it leaves, the next consumer in the colliding consumers for the particular hash ring point will take over.
When a consumer is added, we add 100 marking points to the circle as explained before. Due to the uniform distribution of the hash function, those 100 points act as if the new consumer takes a small slice of keys out of each existing consumer. It maintains the even distribution, on the trade-off that it impacts all existing consumers. This video explains the concept of Consistent Hashing quite well (the only difference is that in Pulsar’s case we used K points instead of K hash functions as noted in the comments)
Sticky
Sticky assumes each consumer is mapped to multiple regions in a range of numbers between 0 to 2^16 (65,536) and there is no overlap between regions. The consumer is selected by running a modulo operation on the message hash by the range size (65,536), the number received (0 <= i < 65,536), is contained within a single region. The consumer mapped to the region is the one selected.
In this algorithm you have full control. Every newly added consumer specifies the ranges it wishes to be mapped to by using Consumer API. When the consumer object is constructed, you can specify the list of ranges. It’s your responsibility to make sure there are no overlaps and all the range is covered by regions.
Example:
Suppose we have 2 consumers (C1 and C2) each specified their ranges, then:
C1 = [0, 16384), [32768, 49152)
C2 = [16384, 32768), [49152, 65536)
0 16,384 32,768 49,152 65,536
|------- C1 ------|------- C2 ------|------- C1 ------|------- C2 ------|
Given a message key Order-3459134
, it’s hash would be murmur32("Order-3459134") = 3112179635
, and it’s index in the range would be 3112179635 mod 65536 = 6067
. That index is contained within [0, 16384)
thus consumer C1 will map to this message key.
If the newly connected consumer didn’t supply their ranges, or they overlap with existing consumer ranges, it’s disconnected, removed from the consumers list and reverted as if it never tried to connect.
How to use mapping algorithms?
To use a mapping algorithm mentioned above, you can specify the Key_Shared Mode when building the consumer:
AUTO_SPLIT
- Auto-split Hash RangeSTICKY
- Sticky
Consistent Hashing will be used instead of Hash Range for Auto-split if the broker configuration subscriptionKeySharedUseConsistentHashing
is enabled.
Preserving order of message delivery by key
In Pulsar 4.0.0, Key_Shared Subscription has been improved to preserve the order of message delivery with the same key when using the AUTO_SPLIT
mode. The message delivery will no longer be blocked completely when new consumers join or leave.
For Key_Shared subscriptions, messages with the same key are delivered and allowed to be in unacknowledged state to only one consumer at a time. This ensures that the order of message delivery by key is preserved.
When new consumers join or leave, the consumer handling a message key can change when the default AUTO_SPLIT
mode is used, but only after all unacknowledged messages for a particular key are acknowledged or the original consumer disconnects.
note
The Key_Shared subscription doesn’t prevent using any methods in the consumer API. For example, the application might call negativeAcknowledge
or the redeliverUnacknowledgedMessages
method. When messages are scheduled for delivery due to these methods, they will get redelivered as soon as possible. There’s no ordering guarantee in these cases, however the guarantee of delivering a message key to a single consumer at a time will continue to be preserved.
Troubleshooting issues when message delivery is blocked for a key in Key_Shared subscriptions AUTO_SPLIT
mode
Pulsar 4.0.0 added consumer-level topic stats to observe unacknowledged messages that block message delivery for a key in Key_Shared subscriptions using the AUTO_SPLIT
mode.
drainingHashesCount
- the current number of hashes in the draining state for this consumerdrainingHashesClearedTotal
- the total number of hashes cleared from the draining state since the consumer connecteddrainingHashesUnackedMessages
- the total number of unacknowledged messages for all draining hashes for this consumerdrainingHashes
- draining hashes information for this consumerhash
- the sticky key hash which is drainingunackMsgs
- the number of unacknowledged messages for this hashblockedAttempts
- the number of times the hash has blocked an attempted delivery of a message
Instead of tracking individual blocked keys, the drainingHashes
field tracks the hashes that are in the draining state and being blocked by unacknowledged messages. The reason to track the hashes instead of the keys is to avoid the overhead of tracking individual keys so that the broker can scale better when there are a large number of keys. The hash space has been reduced to 2^16 (65,536) in Pulsar 4.0.0, down from 2^32 in previous Pulsar versions.
It’s possible to calculate the hash for a key by using the Murmur3 32-bit hash function. The pseudo code to calculate the hash for a key is:
hash = murmur32("key") % 65536 + 1
In addition, the consumer-level topic stats contains the following fields:
keyHashRangeArrays
- the consumer’s hash range assignments in a list of lists where each item contains the start and end as elements.- example
[ [ 2960, 5968 ], [ 22258, 43033 ], [ 49261, 54464 ], [ 55155, 61273 ] ]
- example
This field keyHashRangeArrays
replaces keyHashRange
field available in earlier Pulsar versions. The format of the field is different.
Example of both fields where the difference is visible:
{
"keyHashRangeArrays" : [ [ 2960, 5968 ], [ 22258, 43033 ], [ 49261, 54464 ], [ 55155, 61273 ] ],
"keyHashRanges" : [ "[2960, 5968]", "[22258, 43033]", "[49261, 54464]", "[55155, 61273]" ],
}
The field keyHashRanges
contained the information as a list of string values, which isn’t very usable for most use cases since it would need to be parsed before it can be used.
Example of the consumer stats part of the topic stats for a subscription:
{
"consumers" : [ {
"msgRateOut" : 0.0,
"msgThroughputOut" : 0.0,
"bytesOutCounter" : 1560,
"msgOutCounter" : 30,
"msgRateRedeliver" : 0.0,
"messageAckRate" : 0.0,
"chunkedMessageRate" : 0.0,
"consumerName" : "c1",
"availablePermits" : 70,
"unackedMessages" : 30,
"avgMessagesPerEntry" : 1,
"blockedConsumerOnUnackedMsgs" : false,
"drainingHashesCount" : 5,
"drainingHashesClearedTotal" : 0,
"drainingHashesUnackedMessages" : 10,
"drainingHashes" : [ {
"hash" : 2862,
"unackMsgs" : 2,
"blockedAttempts" : 5
}, {
"hash" : 11707,
"unackMsgs" : 2,
"blockedAttempts" : 9
}, {
"hash" : 15786,
"unackMsgs" : 2,
"blockedAttempts" : 6
}, {
"hash" : 43539,
"unackMsgs" : 2,
"blockedAttempts" : 6
}, {
"hash" : 45436,
"unackMsgs" : 2,
"blockedAttempts" : 9
} ],
"address" : "/127.0.0.1:55829",
"connectedSince" : "2024-10-21T05:39:39.077284+03:00",
"clientVersion" : "Pulsar-Java-v4.0.0",
"lastAckedTimestamp" : 0,
"lastConsumedTimestamp" : 1728527979411,
"lastConsumedFlowTimestamp" : 1728527979106,
"keyHashRangeArrays" : [ [ 2960, 5968 ], [ 22258, 43033 ], [ 49261, 54464 ], [ 55155, 61273 ] ],
"metadata" : { },
"lastAckedTime" : "1970-01-01T02:00:00+02:00",
"lastConsumedTime" : "2024-10-21T05:39:39.411+03:00"
}, {
"msgRateOut" : 0.0,
"msgThroughputOut" : 0.0,
"bytesOutCounter" : 0,
"msgOutCounter" : 0,
"msgRateRedeliver" : 0.0,
"messageAckRate" : 0.0,
"chunkedMessageRate" : 0.0,
"consumerName" : "c2",
"availablePermits" : 1000,
"unackedMessages" : 0,
"avgMessagesPerEntry" : 0,
"blockedConsumerOnUnackedMsgs" : false,
"drainingHashesCount" : 0,
"drainingHashesClearedTotal" : 0,
"drainingHashesUnackedMessages" : 0,
"drainingHashes" : [ ],
"address" : "/127.0.0.1:55829",
"connectedSince" : "2024-10-21T05:39:39.294216+03:00",
"clientVersion" : "Pulsar-Java-v4.0.0",
"lastAckedTimestamp" : 0,
"lastConsumedTimestamp" : 0,
"lastConsumedFlowTimestamp" : 1728527979297,
"keyHashRangeArrays" : [ [ 1, 2959 ], [ 5969, 22257 ], [ 43034, 49260 ], [ 54465, 55154 ], [ 61274, 65535 ] ],
"metadata" : { },
"lastAckedTime" : "1970-01-01T02:00:00+02:00",
"lastConsumedTime" : "1970-01-01T02:00:00+02:00"
} ]
}
Relevant information for consumer c1:
{
"drainingHashesCount" : 5,
"drainingHashesClearedTotal" : 0,
"drainingHashesUnackedMessages" : 10,
"drainingHashes" : [ {
"hash" : 2862,
"unackMsgs" : 2,
"blockedAttempts" : 5
}, {
"hash" : 11707,
"unackMsgs" : 2,
"blockedAttempts" : 9
}, {
"hash" : 15786,
"unackMsgs" : 2,
"blockedAttempts" : 6
}, {
"hash" : 43539,
"unackMsgs" : 2,
"blockedAttempts" : 6
}, {
"hash" : 45436,
"unackMsgs" : 2,
"blockedAttempts" : 9
} ],
}
Relevant information in this case about consumer c2:
{
"keyHashRangeArrays" : [ [ 1, 2959 ], [ 5969, 22257 ], [ 43034, 49260 ], [ 54465, 55154 ], [ 61274, 65535 ] ],
}
In Pulsar 4.0.0, the Key_Shared implementation only blocks hashes that are necessary. For each hash, there is a way to obtain detailed information to determine why the delivery is blocked. The major difference from the previous readPositionWhenJoining
solution is that it is now possible to automate and build CLI and web user interface tools to assist users, making it very easy to troubleshoot issues when message delivery is blocked by unacknowledged messages in Key_Shared AUTO_SPLIT
subscriptions.
A future improvement will be to add a REST API for retrieving the unacknowledged message ID information of the unacknowledged message for a hash. Using this information, it will be possible to find out the details of the message that is blocking a particular hash from being delivered to a consumer. The REST API could also have additional features, such as searching by key or calculating the hash for a given key.
Batching for Key_Shared Subscriptions
note
When the consumers are using the Key_Shared subscription type, you need to disable batching or use key-based batching for the producers.
There are two reasons why the key-based batching is necessary for the Key_Shared subscription type:
- The broker dispatches messages according to the keys of the messages, but the default batching approach might fail to pack the messages with the same key to the same batch.
- Since it is the consumers instead of the broker who dispatch the messages from the batches, the key of the first message in one batch is considered as the key to all messages in this batch, thereby leading to context errors.
The key-based batching aims at resolving the above-mentioned issues. This batching method ensures that the producers pack the messages with the same key to the same batch. The messages without a key are packed into one batch and this batch has no key. When the broker dispatches messages from this batch, it uses NON_KEY
as the key. In addition, each consumer is associated with only one key and should receive only one message batch for the connected key. By default, you can limit batching by configuring the number of messages that producers are allowed to send.
Below are examples of enabling the key-based batching under the Key_Shared subscription type, with client
being the Pulsar client that you created.
- Java
- C++
- Python
Producer<byte[]> producer = client.newProducer()
.topic("my-topic")
.batcherBuilder(BatcherBuilder.KEY_BASED)
.create();
ProducerConfiguration producerConfig;
producerConfig.setBatchingType(ProducerConfiguration::BatchingType::KeyBasedBatching);
Producer producer;
client.createProducer("my-topic", producerConfig, producer);
producer = client.create_producer(topic='my-topic', batching_type=pulsar.BatchingType.KeyBased)
note
When you use Key_Shared subscriptions, be aware that:
- You need to specify a key or ordering key for messages.
- You cannot use cumulative acknowledgment.
- When the position of the newest message in a topic is
X
, a key_shared consumer that is newly attached to the same subscription and connected to the topic will not receive any messages until all the messages beforeX
have been acknowledged.
Subscription modes
What is a subscription mode
The subscription mode indicates the cursor belongs to durable type or non-durable type.
When a subscription is created, an associated cursor is created to record the last consumed position.
When a consumer of the subscription restarts, it can continue consuming from the last message it consumes.
Subscription mode | Description | Note |
---|---|---|
Durable | The cursor is durable, which retains messages and persists the current position. If a broker restarts from a failure, it can recover the cursor from the persistent storage (BookKeeper), so that messages can continue to be consumed from the last consumed position. | Durable is the default subscription mode. |
NonDurable | The cursor is non-durable. Once a broker stops, the cursor is lost and can never be recovered, so that messages can not continue to be consumed from the last consumed position. | Reader’s subscription mode is NonDurable in nature and it does not prevent data in a topic from being deleted. Reader’s subscription mode can not be changed. |
A subscription can have one or more consumers. When a consumer subscribes to a topic, it must specify the subscription name. A durable subscription and a non-durable subscription can have the same name, they are independent of each other. If a consumer specifies a subscription that does not exist before, the subscription is automatically created.
When to use
By default, messages of a topic without any durable subscriptions are marked as deleted. If you want to prevent the messages from being marked as deleted, you can create a durable subscription for this topic. In this case, only acknowledged messages are marked as deleted. For more information, see message retention and expiry.
How to use
After a consumer is created, the default subscription mode of the consumer is Durable
. You can change the subscription mode to NonDurable
by making changes to the consumer’s configuration.
- Durable
- Non-durable
Consumer<byte[]> consumer = pulsarClient.newConsumer()
.topic("my-topic")
.subscriptionName("my-sub")
.subscriptionMode(SubscriptionMode.Durable)
.subscribe();
Consumer<byte[]> consumer = pulsarClient.newConsumer()
.topic("my-topic")
.subscriptionName("my-sub")
.subscriptionMode(SubscriptionMode.NonDurable)
.subscribe();
For how to create, check, or delete a durable subscription, see manage subscriptions.
Multi-topic subscriptions
When a consumer subscribes to a Pulsar topic, by default it subscribes to one specific topic, such as persistent://public/default/my-topic
. As of Pulsar version 1.23.0-incubating, however, Pulsar consumers can simultaneously subscribe to multiple topics. You can define a list of topics in two ways:
- On the basis of a regular expression (regex), for example,
persistent://public/default/finance-.*
- By explicitly defining a list of topics
note
When subscribing to multiple topics by regex, all topics must be in the same namespace.
When subscribing to multiple topics, the Pulsar client automatically makes a call to the Pulsar API to discover the topics that match the regex pattern/list, and then subscribe to all of them. If any of the topics do not exist, the consumer auto-subscribes to them once the topics are created.
note
No ordering guarantees across multiple topics When a producer sends messages to a single topic, all messages are guaranteed to be read from that topic in the same order. However, these guarantees do not hold across multiple topics. So when a producer sends messages to multiple topics, the order in which messages are read from those topics is not guaranteed to be the same.
The following are multi-topic subscription examples for Java.
import java.util.regex.Pattern;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.PulsarClient;
PulsarClient pulsarClient = // Instantiate Pulsar client object
// Subscribe to all topics in a namespace
Pattern allTopicsInNamespace = Pattern.compile("persistent://public/default/.*");
Consumer<byte[]> allTopicsConsumer = pulsarClient.newConsumer()
.topicsPattern(allTopicsInNamespace)
.subscriptionName("subscription-1")
.subscribe();
// Subscribe to a subsets of topics in a namespace, based on regex
Pattern someTopicsInNamespace = Pattern.compile("persistent://public/default/foo.*");
Consumer<byte[]> someTopicsConsumer = pulsarClient.newConsumer()
.topicsPattern(someTopicsInNamespace)
.subscriptionName("subscription-1")
.subscribe();
For code examples, see Java.
Partitioned topics
Normal topics are served only by a single broker, which limits the maximum throughput of the topic. Partitioned topic is a special type of topic handled by multiple brokers, thus allowing for higher throughput.
A partitioned topic is implemented as N internal topics, where N is the number of partitions. When publishing messages to a partitioned topic, each message is routed to one of several brokers. The distribution of partitions across brokers is handled automatically by Pulsar.
The diagram below illustrates this:
The Topic1 topic has five partitions (P0 through P4) split across three brokers. Because there are more partitions than brokers, two brokers handle two partitions a piece, while the third handles only one (again, Pulsar handles this distribution of partitions automatically).
Messages for this topic are broadcast to two consumers. The routing mode determines each message should be published to which partition, while the subscription type determines which messages go to which consumers.
Decisions about routing and subscription modes can be made separately in most cases. In general, throughput concerns should guide partitioning/routing decisions while subscription decisions should be guided by application semantics.
There is no difference between partitioned topics and normal topics in terms of how subscription types work, as partitioning only determines what happens between when a message is published by a producer and processed and acknowledged by a consumer.
Partitioned topics need to be explicitly created via the admin API. The number of partitions can be specified when creating the topic.
Routing modes
When publishing to partitioned topics, you must specify a routing mode. The routing mode determines each message should be published to which partition or which internal topic.
There are three MessageRoutingMode available:
Mode | Description |
---|---|
RoundRobinPartition | If no key is provided, the producer will publish messages across all partitions in round-robin fashion to achieve maximum throughput. Please note that round-robin is not done per individual message but rather it’s set to the same boundary of batching delay, to ensure batching is effective. While if a key is specified on the message, the partitioned producer will hash the key and assign message to a particular partition. This is the default mode. |
SinglePartition | If no key is provided, the producer will randomly pick one single partition and publish all the messages into that partition. While if a key is specified on the message, the partitioned producer will hash the key and assign message to a particular partition. |
CustomPartition | Use custom message router implementation that will be called to determine the partition for a particular message. User can create a custom routing mode by using the Java client and implementing the MessageRouter interface. |
Ordering guarantee
The ordering of messages is related to MessageRoutingMode and Message Key. Usually, user would want an ordering of Per-key-partition guarantee.
If there is a key attached to message, the messages will be routed to corresponding partitions based on the hashing scheme specified by HashingScheme in ProducerBuilder, when using either SinglePartition
or RoundRobinPartition
mode.
Ordering guarantee | Description | Routing Mode and Key |
---|---|---|
Per-key-partition | All the messages with the same key will be in order and be placed in same partition. | Use either SinglePartition or RoundRobinPartition mode, and Key is provided by each message. |
Per-producer | All the messages from the same producer will be in order. | Use SinglePartition mode, and no Key is provided for each message. |
Hashing scheme
Hashing Scheme is an enum that represents sets of standard hashing functions available when choosing the partition to use for a particular message.
There are 2 types of standard hashing functions available:
- JavaStringHash
- Murmur3_32Hash
The default hashing function for producers is JavaStringHash
. Please pay attention that JavaStringHash
is not useful when producers can be from different multiple language clients, under this use case, it is recommended to use Murmur3_32Hash
.
Non-persistent topics
By default, Pulsar persistently stores all unacknowledged messages on multiple BookKeeper bookies (storage nodes). Data for messages on persistent topics can thus survive broker restarts and subscriber failover.
Pulsar also, however, supports non-persistent topics. Non-persistent topics are Pulsar topics in which message data is never persistently stored to disk and kept only in memory. When using non-persistent delivery, killing a Pulsar broker or disconnecting a subscriber to a topic means that all in-transit messages are lost on that (non-persistent) topic, meaning that clients may see message loss.
Non-persistent topics have names of this form (note the non-persistent
in the name):
non-persistent://tenant/namespace/topic
For more info on using non-persistent topics, see the Non-persistent messaging cookbook.
In non-persistent topics, brokers immediately deliver messages to all connected subscribers without persisting them in BookKeeper. If a subscriber is disconnected, the broker will not be able to deliver those in-transit messages, and subscribers will never be able to receive those messages again. Eliminating the persistent storage step makes messaging on non-persistent topics slightly faster than on persistent topics in some cases, but with the caveat that some core benefits of Pulsar are lost.
With non-persistent topics, message data lives only in memory, without a specific buffer - which means data is not buffered in memory. The received messages are immediately transmitted to all connected consumers. If a message broker fails or message data can otherwise not be retrieved from memory, your message data may be lost. Use non-persistent topics only if you’re certain that your use case requires it and can sustain it.
By default, non-persistent topics are enabled on Pulsar brokers. You can disable them in the broker’s configuration. You can manage non-persistent topics using the pulsar-admin topics
command. For more information, see pulsar-admin.
Currently, non-persistent topics which are not partitioned are not persisted to ZooKeeper, which means if the broker owning them crashes, they do not get re-assigned to another broker because they only exist in the owner broker memory. The current workaround is to set the value of allowAutoTopicCreation
to true
and allowAutoTopicCreationType
to non-partitioned
(they are default values) in broker configuration.
Performance
With persistent topics, all messages are durably persisted on disks, whereas with non-persistent topics, brokers don’t persist messages and immediately send acks back to the producer as soon as that message is delivered to connected brokers, so non-persistent messaging is usually faster than persistent messaging. Producers thus see comparatively low publish latency with non-persistent topics.
Client API
Producers and consumers can connect to non-persistent topics in the same way as persistent topics, with the crucial difference that the topic name must start with non-persistent
. All the subscription types—-exclusive, shared, key_shared and failover-—are supported for non-persistent topics.
Here’s an example Java consumer for a non-persistent topic:
PulsarClient client = PulsarClient.builder()
.serviceUrl("pulsar://localhost:6650")
.build();
String npTopic = "non-persistent://public/default/my-topic";
String subscriptionName = "my-subscription-name";
Consumer<byte[]> consumer = client.newConsumer()
.topic(npTopic)
.subscriptionName(subscriptionName)
.subscribe();
Here’s an example Java producer for the same non-persistent topic:
Producer<byte[]> producer = client.newProducer()
.topic(npTopic)
.create();
System topic
System topic is a predefined topic for internal use within Pulsar. It can be either a persistent or non-persistent topic.
System topics serve to implement certain features and eliminate dependencies on third-party components, such as transactions, heartbeat detections, topic-level policies, and resource group services. System topics empower the implementation of these features to be simplified, dependent, and flexible. Take heartbeat detections for example, you can leverage the system topic for health check to internally enable producer/reader to produce/consume messages under the heartbeat namespace, which can detect whether the current service is still alive.
The following table outlines the available system topics for each specific namespace.
Namespace | TopicName | Domain | Count | Usage |
---|---|---|---|---|
pulsar/system | transactioncoordinator_assign\${id} | Persistent | Default 16 | Transaction coordinator |
pulsar/system | transactionlog\${tc_id} | Persistent | Default 16 | Transaction log |
pulsar/system | resource-usage | Non-persistent | Default 4 | Resource group service |
host/port | heartbeat | Persistent | 1 | Heartbeat detection |
User-defined-ns | change_events | Persistent | Default 4 | Topic events |
User-defined-ns | transaction_buffer_snapshot | Persistent | One per namespace | Transaction buffer snapshots |
User-defined-ns | \${topicName}transaction_pending_ack | Persistent | One per every topic subscription acknowledged with transactions | Acknowledgments with transactions |
note
You cannot create any system topics. To list system topics, you can add the option
--include-system-topic
when you get the topic list by using Pulsar admin API.Since Pulsar version 2.11.0, system topics are enabled by default. In earlier versions, you need to change the following configurations in the
conf/broker.conf
orconf/standalone.conf
file to enable system topics.systemTopicEnabled=true
topicLevelPoliciesEnabled=true
Message redelivery
Apache Pulsar supports graceful failure handling and ensures critical data is not lost. Software will always have unexpected conditions and at times messages may not be delivered successfully. Therefore, it is important to have a built-in mechanism that handles failure, particularly in asynchronous messaging as highlighted in the following examples.
- Consumers get disconnected from the database or the HTTP server. When this happens, the database is temporarily offline while the consumer is writing the data to it and the external HTTP server that the consumer calls are momentarily unavailable.
- Consumers get disconnected from a broker due to consumer crashes, broken connections, etc. As a consequence, unacknowledged messages are delivered to other available consumers.
Message redelivery in Apache Pulsar avoids failure in asynchronous messaging and other message delivery failures using at-least-once delivery semantics that ensure Pulsar processes a message more than once.
To utilize message redelivery, you need to enable this mechanism before the broker can resend the unacknowledged messages in Apache Pulsar client. You can activate the message redelivery mechanism in Apache Pulsar using three methods.
Message retention and expiry
By default, Pulsar message brokers:
- immediately delete all messages that have been acknowledged by a consumer, and
- persistently store all unacknowledged messages in a message backlog.
Pulsar has two features, however, that enable you to override this default behavior:
- Message retention enables you to store messages that have been acknowledged by a consumer
- Message expiry enables you to set a time to live (TTL) for messages that have not yet been acknowledged
tip
All message retention and expiry are managed at the namespace level. For a how-to, see the Message retention and expiry cookbook.
With message retention, shown at the top, a retention policy applied to all topics in a namespace dictates that some messages are durably stored in Pulsar even though they’ve already been acknowledged. Acknowledged messages that are not covered by the retention policy are deleted. Without a retention policy, all of the acknowledged messages would be deleted.
With message expiry, shown at the bottom, some messages are deleted, even though they haven’t been acknowledged, because they’ve expired according to the TTL applied to the namespace (for example because a TTL of 5 minutes has been applied and the messages haven’t been acknowledged but are 10 minutes old).
Message deduplication
Message duplication occurs when a message is persisted by Pulsar more than once. Message deduplication ensures that each message produced on Pulsar topics is persisted to disk only once, even if the message is produced more than once. Message deduplication is handled automatically on the server side.
The following diagram illustrates what happens when message deduplication is disabled vs. enabled:
Message deduplication is disabled in the scenario shown at the top. Here, a producer publishes message 1 on a topic; the message reaches a Pulsar broker and is persisted to BookKeeper. The producer then sends message 1 again (in this case due to some retry logic), and the message is received by the broker and stored in BookKeeper again, which means that duplication has occurred.
In the second scenario at the bottom, the producer publishes message 1, which is received by the broker and persisted, as in the first scenario. When the producer attempts to publish the message again, however, the broker knows that it has already seen message 1 and thus does not persist the message.
tip
- Message deduplication is handled at the namespace level or the topic level. For more instructions, see the message deduplication cookbook.
- You can read the design of Message Deduplication in PIP-6.
Producer idempotency
The other available approach to message deduplication is producer idempotency, which means each message is only produced once without data loss and duplication. The drawback of this approach is that it defers the work of message deduplication to the application. In Pulsar, this is handled at the broker level, so you do not need to modify your Pulsar client code. Instead, you only need to make administrative changes. For details, see Managing message deduplication.
Deduplication and effectively-once semantics
Message deduplication makes Pulsar an ideal messaging system to be used in conjunction with stream processing engines (SPEs) and other systems seeking to provide effectively-once processing semantics. Messaging systems that do not offer automatic message deduplication require the SPE or other system to guarantee deduplication, which means that strict message ordering comes at the cost of burdening the application with the responsibility of deduplication. With Pulsar, strict ordering guarantees come at no application-level cost.
Delayed message delivery
Delayed message delivery enables you to consume a message later. In this mechanism, a message is stored in BookKeeper. The DelayedDeliveryTracker
maintains the time index (time -> messageId) in memory after the message is published to a broker. This message will be delivered to a consumer once the specified delay is over.
note
Only shared and key-shared subscriptions support delayed message delivery. In other subscriptions, delayed messages are dispatched immediately.
The diagram below illustrates the concept of delayed message delivery:
A broker saves a message without any check. When a consumer consumes a message, if the message is set to delay, then the message is added to DelayedDeliveryTracker
. A subscription checks and gets timeout messages from DelayedDeliveryTracker
.
note
Work with retention policy: In Pulsar, the ledger will be deleted automatically after the messages in this ledger have been consumed. Pulsar will delete the front ledgers of a topic but will not delete ledgers from the middle of a topic. It means that if you send a message that is delayed for a long time, the message will not be consumed until it reaches the delay time. This means all the ledgers on this topic could not be deleted until the delayed message is consumed, even if some subsequent ledgers are fully consumed.
Work with backlog quota policy: After using delayed messages, it is advisable to exercise caution when using the Backlog Quota strategy. This is because delayed messages can result in not being consumed for an extended period, triggering the Backlog Quota strategy and causing subsequent message sends to be rejected.
Work with backlog TTL policy: When the TTL expires, Pulsar automatically moves the message to the acknowledged state (and thus makes it ready for deletion) even if the messages are delayed messages and does not care about when the expected delayed time is.
Broker
Delayed message delivery is enabled by default. You can change it in the broker configuration file as below:
# Whether to enable the delayed delivery for messages.
# If disabled, messages are immediately delivered and there is no tracking overhead.
delayedDeliveryEnabled=true
# Control the ticking time for the retry of delayed message delivery,
# affecting the accuracy of the delivery time compared to the scheduled time.
# Note that this time is used to configure the HashedWheelTimer's tick time for the
# InMemoryDelayedDeliveryTrackerFactory (the default DelayedDeliverTrackerFactory).
# Default is 1 second.
delayedDeliveryTickTimeMillis=1000
# When using the InMemoryDelayedDeliveryTrackerFactory (the default DelayedDeliverTrackerFactory), whether
# the deliverAt time is strictly followed. When false (default), messages may be sent to consumers before the deliverAt
# time by as much as the tickTimeMillis. This can reduce the overhead on the broker of maintaining the delayed index
# for a potentially very short time period. When true, messages will not be sent to consumer until the deliverAt time
# has passed, and they may be as late as the deliverAt time plus the tickTimeMillis for the topic plus the
# delayedDeliveryTickTimeMillis.
isDelayedDeliveryDeliverAtTimeStrict=false
Producer
The following is an example of delayed message delivery for a producer in Java:
// message to be delivered at the configured delay interval
producer.newMessage().deliverAfter(3L, TimeUnit.Minute).value("Hello Pulsar!").send();