Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

fix: seek race #1265

Merged
merged 4 commits into from
Nov 4, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
22 changes: 12 additions & 10 deletions pulsar/consumer_impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -712,22 +712,29 @@ func (c *consumer) Seek(msgID MessageID) error {
return err
}

if err := c.consumers[msgID.PartitionIdx()].Seek(msgID); err != nil {
return err
}

consumer := c.consumers[msgID.PartitionIdx()]
consumer.pauseDispatchMessage()
// clear messageCh
for len(c.messageCh) > 0 {
<-c.messageCh
}

return nil
return consumer.Seek(msgID)
}

func (c *consumer) SeekByTime(time time.Time) error {
c.Lock()
defer c.Unlock()
var errs error

for _, cons := range c.consumers {
cons.pauseDispatchMessage()
}
// clear messageCh
for len(c.messageCh) > 0 {
<-c.messageCh
}

// run SeekByTime on every partition of topic
for _, cons := range c.consumers {
if err := cons.SeekByTime(time); err != nil {
Expand All @@ -736,11 +743,6 @@ func (c *consumer) SeekByTime(time time.Time) error {
}
}

// clear messageCh
for len(c.messageCh) > 0 {
<-c.messageCh
}

return errs
}

Expand Down
109 changes: 64 additions & 45 deletions pulsar/consumer_partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"math"
"strings"
"sync"
"sync/atomic"
"time"

"github.com/apache/pulsar-client-go/pulsar/backoff"
Expand Down Expand Up @@ -185,6 +186,16 @@ type partitionConsumer struct {

redirectedClusterURI string
backoffPolicyFunc func() backoff.Policy

dispatcherSeekingControlCh chan struct{}
isSeeking atomic.Bool
}

// pauseDispatchMessage used to discard the message in the dispatcher goroutine.
// This method will be called When the parent consumer performs the seek operation.
// After the seek operation, the dispatcher will continue dispatching messages automatically.
func (pc *partitionConsumer) pauseDispatchMessage() {
pc.dispatcherSeekingControlCh <- struct{}{}
}

func (pc *partitionConsumer) ActiveConsumerChanged(isActive bool) {
Expand Down Expand Up @@ -329,27 +340,28 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon
}

pc := &partitionConsumer{
parentConsumer: parent,
client: client,
options: options,
topic: options.topic,
name: options.consumerName,
consumerID: client.rpcClient.NewConsumerID(),
partitionIdx: int32(options.partitionIdx),
eventsCh: make(chan interface{}, 10),
maxQueueSize: int32(options.receiverQueueSize),
queueCh: make(chan *message, options.receiverQueueSize),
startMessageID: atomicMessageID{msgID: options.startMessageID},
connectedCh: make(chan struct{}),
messageCh: messageCh,
connectClosedCh: make(chan *connectionClosed, 1),
closeCh: make(chan struct{}),
clearQueueCh: make(chan func(id *trackingMessageID)),
compressionProviders: sync.Map{},
dlq: dlq,
metrics: metrics,
schemaInfoCache: newSchemaInfoCache(client, options.topic),
backoffPolicyFunc: boFunc,
parentConsumer: parent,
client: client,
options: options,
topic: options.topic,
name: options.consumerName,
consumerID: client.rpcClient.NewConsumerID(),
partitionIdx: int32(options.partitionIdx),
eventsCh: make(chan interface{}, 10),
maxQueueSize: int32(options.receiverQueueSize),
queueCh: make(chan *message, options.receiverQueueSize),
startMessageID: atomicMessageID{msgID: options.startMessageID},
connectedCh: make(chan struct{}),
messageCh: messageCh,
connectClosedCh: make(chan *connectionClosed, 1),
closeCh: make(chan struct{}),
clearQueueCh: make(chan func(id *trackingMessageID)),
compressionProviders: sync.Map{},
dlq: dlq,
metrics: metrics,
schemaInfoCache: newSchemaInfoCache(client, options.topic),
backoffPolicyFunc: boFunc,
dispatcherSeekingControlCh: make(chan struct{}),
}
if pc.options.autoReceiverQueueSize {
pc.currentQueueSize.Store(initialReceiverQueueSize)
Expand Down Expand Up @@ -1440,17 +1452,20 @@ func (pc *partitionConsumer) dispatcher() {
}
nextMessageSize = queueMsg.size()

if pc.dlq.shouldSendToDlq(&nextMessage) {
// pass the message to the DLQ router
pc.metrics.DlqCounter.Inc()
messageCh = pc.dlq.Chan()
if !pc.isSeeking.Load() {
crossoverJie marked this conversation as resolved.
Show resolved Hide resolved
if pc.dlq.shouldSendToDlq(&nextMessage) {
// pass the message to the DLQ router
pc.metrics.DlqCounter.Inc()
messageCh = pc.dlq.Chan()
} else {
// pass the message to application channel
messageCh = pc.messageCh
}
pc.metrics.PrefetchedMessages.Dec()
pc.metrics.PrefetchedBytes.Sub(float64(len(queueMsg.payLoad)))
crossoverJie marked this conversation as resolved.
Show resolved Hide resolved
} else {
// pass the message to application channel
messageCh = pc.messageCh
pc.log.Debug("skip dispatching messages when seeking")
}

pc.metrics.PrefetchedMessages.Dec()
pc.metrics.PrefetchedBytes.Sub(float64(len(queueMsg.payLoad)))
} else {
queueCh = pc.queueCh
}
Expand Down Expand Up @@ -1483,6 +1498,13 @@ func (pc *partitionConsumer) dispatcher() {
pc.log.WithError(err).Error("unable to send initial permits to broker")
}

case _, ok := <-pc.dispatcherSeekingControlCh:
if !ok {
return
}
pc.log.Debug("received dispatcherSeekingControlCh, set isSeek to true")
pc.isSeeking.Store(true)

case msg, ok := <-queueCh:
if !ok {
return
Expand Down Expand Up @@ -1587,22 +1609,16 @@ func (pc *partitionConsumer) runEventsLoop() {
}()
pc.log.Debug("get into runEventsLoop")

go func() {
for {
select {
case <-pc.closeCh:
pc.log.Info("close consumer, exit reconnect")
return
case connectionClosed := <-pc.connectClosedCh:
pc.log.Debug("runEventsLoop will reconnect")
pc.reconnectToBroker(connectionClosed)
}
}
}()

for {
for i := range pc.eventsCh {
switch v := i.(type) {
select {
case <-pc.closeCh:
pc.log.Info("close consumer, exit reconnect")
return
case connectionClosed := <-pc.connectClosedCh:
pc.log.Debug("runEventsLoop will reconnect")
pc.reconnectToBroker(connectionClosed)
case event := <-pc.eventsCh:
switch v := event.(type) {
case *ackRequest:
pc.internalAck(v)
case *ackWithTxnRequest:
Expand Down Expand Up @@ -1680,6 +1696,9 @@ func (pc *partitionConsumer) internalClose(req *closeRequest) {
}

func (pc *partitionConsumer) reconnectToBroker(connectionClosed *connectionClosed) {
if pc.isSeeking.CompareAndSwap(true, false) {
crossoverJie marked this conversation as resolved.
Show resolved Hide resolved
pc.log.Debug("seek operation triggers reconnection, and reset isSeeking")
}
var (
maxRetry int
delayReconnectTime, totalDelayReconnectTime time.Duration
Expand Down
Loading