Skip to content

Commit

Permalink
[feat] Support partitioned topic reader (#1178)
Browse files Browse the repository at this point in the history
Master Issue: #1177

### Motivation

Currently, there is an issue with the reader implementation. If the reader is creating, it won't get the topic metadata from the topic. The reader can only read messages from a single topic. If the topic is a partitioned topic, the reader won't know that and will try to create a non-partition topic with the same name. And it will lead to this issue: apache/pulsar#22032

### Modifications

- Support partitioned topic reader

(cherry picked from commit 3b9b1f8)
  • Loading branch information
RobertIndie committed Feb 29, 2024
1 parent 68cd09a commit 6ce5421
Show file tree
Hide file tree
Showing 6 changed files with 227 additions and 76 deletions.
7 changes: 7 additions & 0 deletions pulsar/consumer.go
Original file line number Diff line number Diff line change
Expand Up @@ -246,6 +246,13 @@ type ConsumerOptions struct {
// SubscriptionMode specifies the subscription mode to be used when subscribing to a topic.
// Default is `Durable`
SubscriptionMode SubscriptionMode

// StartMessageIDInclusive, if true, the consumer will start at the `StartMessageID`, included.
// Default is `false` and the consumer will start from the "next" message
StartMessageIDInclusive bool

// startMessageID specifies the message id to start from. Currently, it's only used for the reader internally.
startMessageID *trackingMessageID
}

// Consumer is an interface that abstracts behavior of Pulsar's consumer
Expand Down
47 changes: 46 additions & 1 deletion pulsar/consumer_impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -384,7 +384,8 @@ func (c *consumer) internalTopicSubscribeToPartitions() error {
metadata: metadata,
subProperties: subProperties,
replicateSubscriptionState: c.options.ReplicateSubscriptionState,
startMessageID: nil,
startMessageID: c.options.startMessageID,
startMessageIDInclusive: c.options.StartMessageIDInclusive,
subscriptionMode: c.options.SubscriptionMode,
readCompacted: c.options.ReadCompacted,
interceptors: c.options.Interceptors,
Expand Down Expand Up @@ -707,6 +708,50 @@ func (c *consumer) checkMsgIDPartition(msgID MessageID) error {
return nil
}

func (c *consumer) hasNext() bool {
ctx, cancel := context.WithCancel(context.Background())
defer cancel() // Make sure all paths cancel the context to avoid context leak

var wg sync.WaitGroup
wg.Add(len(c.consumers))

hasNext := make(chan bool)
for _, pc := range c.consumers {
pc := pc
go func() {
defer wg.Done()
if pc.hasNext() {
select {
case hasNext <- true:
case <-ctx.Done():
}
}
}()
}

go func() {
wg.Wait()
close(hasNext) // Close the channel after all goroutines have finished
}()

// Wait for either a 'true' result or for all goroutines to finish
for hn := range hasNext {
if hn {
return true
}
}

return false
}

func (c *consumer) setLastDequeuedMsg(msgID MessageID) error {
if err := c.checkMsgIDPartition(msgID); err != nil {
return err
}
c.consumers[msgID.PartitionIdx()].lastDequeuedMsg = toTrackingMessageID(msgID)
return nil
}

var r = &random{
R: rand.New(rand.NewSource(time.Now().UnixNano())),
}
Expand Down
36 changes: 36 additions & 0 deletions pulsar/consumer_partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -174,6 +174,8 @@ type partitionConsumer struct {
chunkedMsgCtxMap *chunkedMsgCtxMap
unAckChunksTracker *unAckChunksTracker
ackGroupingTracker ackGroupingTracker

lastMessageInBroker *trackingMessageID
}

func (pc *partitionConsumer) ActiveConsumerChanged(isActive bool) {
Expand Down Expand Up @@ -1970,6 +1972,40 @@ func (pc *partitionConsumer) discardCorruptedMessage(msgID *pb.MessageIdData,
pc.availablePermits.inc()
}

func (pc *partitionConsumer) hasNext() bool {
if pc.lastMessageInBroker != nil && pc.hasMoreMessages() {
return true
}

for {
lastMsgID, err := pc.getLastMessageID()
if err != nil {
pc.log.WithError(err).Error("Failed to get last message id from broker")
continue
} else {
pc.lastMessageInBroker = lastMsgID
break
}
}

return pc.hasMoreMessages()
}

func (pc *partitionConsumer) hasMoreMessages() bool {
if pc.lastDequeuedMsg != nil {
return pc.lastMessageInBroker.isEntryIDValid() && pc.lastMessageInBroker.greater(pc.lastDequeuedMsg.messageID)
}

if pc.options.startMessageIDInclusive {
return pc.lastMessageInBroker.isEntryIDValid() &&
pc.lastMessageInBroker.greaterEqual(pc.startMessageID.get().messageID)
}

// Non-inclusive
return pc.lastMessageInBroker.isEntryIDValid() &&
pc.lastMessageInBroker.greater(pc.startMessageID.get().messageID)
}

// _setConn sets the internal connection field of this partition consumer atomically.
// Note: should only be called by this partition consumer when a new connection is available.
func (pc *partitionConsumer) _setConn(conn internal.Connection) {
Expand Down
1 change: 1 addition & 0 deletions pulsar/reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,5 +136,6 @@ type Reader interface {
SeekByTime(time time.Time) error

// GetLastMessageID get the last message id available for consume.
// It only works for single topic reader. It will return an error when the reader is the multi-topic reader.
GetLastMessageID() (MessageID, error)
}
112 changes: 47 additions & 65 deletions pulsar/reader_impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,12 +34,11 @@ const (

type reader struct {
sync.Mutex
client *client
pc *partitionConsumer
messageCh chan ConsumerMessage
lastMessageInBroker *trackingMessageID
log log.Logger
metrics *internal.LeveledMetrics
client *client
messageCh chan ConsumerMessage
log log.Logger
metrics *internal.LeveledMetrics
c *consumer
}

func newReader(client *client, options ReaderOptions) (Reader, error) {
Expand Down Expand Up @@ -98,25 +97,25 @@ func newReader(client *client, options ReaderOptions) (Reader, error) {
options.ExpireTimeOfIncompleteChunk = time.Minute
}

consumerOptions := &partitionConsumerOpts{
topic: options.Topic,
consumerName: options.Name,
subscription: subscriptionName,
subscriptionType: Exclusive,
receiverQueueSize: receiverQueueSize,
consumerOptions := &ConsumerOptions{
Topic: options.Topic,
Name: options.Name,
SubscriptionName: subscriptionName,
Type: Exclusive,
ReceiverQueueSize: receiverQueueSize,
SubscriptionMode: NonDurable,
ReadCompacted: options.ReadCompacted,
Properties: options.Properties,
NackRedeliveryDelay: defaultNackRedeliveryDelay,
ReplicateSubscriptionState: false,
Decryption: options.Decryption,
Schema: options.Schema,
BackoffPolicy: options.BackoffPolicy,
MaxPendingChunkedMessage: options.MaxPendingChunkedMessage,
ExpireTimeOfIncompleteChunk: options.ExpireTimeOfIncompleteChunk,
AutoAckIncompleteChunk: options.AutoAckIncompleteChunk,
startMessageID: startMessageID,
startMessageIDInclusive: options.StartMessageIDInclusive,
subscriptionMode: NonDurable,
readCompacted: options.ReadCompacted,
metadata: options.Properties,
nackRedeliveryDelay: defaultNackRedeliveryDelay,
replicateSubscriptionState: false,
decryption: options.Decryption,
schema: options.Schema,
backoffPolicy: options.BackoffPolicy,
maxPendingChunkedMessage: options.MaxPendingChunkedMessage,
expireTimeOfIncompleteChunk: options.ExpireTimeOfIncompleteChunk,
autoAckIncompleteChunk: options.AutoAckIncompleteChunk,
StartMessageIDInclusive: options.StartMessageIDInclusive,
}

reader := &reader{
Expand All @@ -131,20 +130,25 @@ func newReader(client *client, options ReaderOptions) (Reader, error) {
if err != nil {
return nil, err
}
// Provide dummy rlq router with not dlq policy
rlq, err := newRetryRouter(client, nil, false, client.log)
if err != nil {
return nil, err
}

pc, err := newPartitionConsumer(nil, client, consumerOptions, reader.messageCh, dlq, reader.metrics)
c, err := newInternalConsumer(client, *consumerOptions, options.Topic, reader.messageCh, dlq, rlq, false)
if err != nil {
close(reader.messageCh)
return nil, err
}
reader.c = c

reader.pc = pc
reader.metrics.ReadersOpened.Inc()
return reader, nil
}

func (r *reader) Topic() string {
return r.pc.topic
return r.c.topic
}

func (r *reader) Next(ctx context.Context) (Message, error) {
Expand All @@ -158,9 +162,14 @@ func (r *reader) Next(ctx context.Context) (Message, error) {
// Acknowledge message immediately because the reader is based on non-durable subscription. When it reconnects,
// it will specify the subscription position anyway
msgID := cm.Message.ID()
mid := toTrackingMessageID(msgID)
r.pc.lastDequeuedMsg = mid
r.pc.AckID(mid)
err := r.c.setLastDequeuedMsg(msgID)
if err != nil {
return nil, err
}
err = r.c.AckID(msgID)
if err != nil {
return nil, err
}
return cm.Message, nil
case <-ctx.Done():
return nil, ctx.Err()
Expand All @@ -169,41 +178,11 @@ func (r *reader) Next(ctx context.Context) (Message, error) {
}

func (r *reader) HasNext() bool {
if r.lastMessageInBroker != nil && r.hasMoreMessages() {
return true
}

for {
lastMsgID, err := r.pc.getLastMessageID()
if err != nil {
r.log.WithError(err).Error("Failed to get last message id from broker")
continue
} else {
r.lastMessageInBroker = lastMsgID
break
}
}

return r.hasMoreMessages()
}

func (r *reader) hasMoreMessages() bool {
if r.pc.lastDequeuedMsg != nil {
return r.lastMessageInBroker.isEntryIDValid() && r.lastMessageInBroker.greater(r.pc.lastDequeuedMsg.messageID)
}

if r.pc.options.startMessageIDInclusive {
return r.lastMessageInBroker.isEntryIDValid() &&
r.lastMessageInBroker.greaterEqual(r.pc.startMessageID.get().messageID)
}

// Non-inclusive
return r.lastMessageInBroker.isEntryIDValid() &&
r.lastMessageInBroker.greater(r.pc.startMessageID.get().messageID)
return r.c.hasNext()
}

func (r *reader) Close() {
r.pc.Close()
r.c.Close()
r.client.handlers.Del(r)
r.metrics.ReadersClosed.Inc()
}
Expand Down Expand Up @@ -235,16 +214,19 @@ func (r *reader) Seek(msgID MessageID) error {
return nil
}

return r.pc.Seek(mid)
return r.c.Seek(mid)
}

func (r *reader) SeekByTime(time time.Time) error {
r.Lock()
defer r.Unlock()

return r.pc.SeekByTime(time)
return r.c.SeekByTime(time)
}

func (r *reader) GetLastMessageID() (MessageID, error) {
return r.pc.getLastMessageID()
if len(r.c.consumers) > 1 {
return nil, fmt.Errorf("GetLastMessageID is not supported for multi-topics reader")
}
return r.c.consumers[0].getLastMessageID()
}

0 comments on commit 6ce5421

Please sign in to comment.