@@ -19,6 +19,7 @@ package pulsar
1919
2020import (
2121 "container/list"
22+ "context"
2223 "encoding/hex"
2324 "fmt"
2425 "math"
@@ -612,7 +613,6 @@ func (pc *partitionConsumer) getLastMessageID() (*trackingMessageID, error) {
612613 pc .log .WithField ("state" , state ).Error ("Failed to getLastMessageID for the closing or closed consumer" )
613614 return nil , errors .New ("failed to getLastMessageID for the closing or closed consumer" )
614615 }
615- remainTime := pc .client .operationTimeout
616616 bo := pc .backoffPolicyFunc ()
617617 request := func () (* trackingMessageID , error ) {
618618 req := & getLastMsgIDRequest {doneCh : make (chan struct {})}
@@ -622,23 +622,20 @@ func (pc *partitionConsumer) getLastMessageID() (*trackingMessageID, error) {
622622 <- req .doneCh
623623 return req .msgID , req .err
624624 }
625- for {
626- msgID , err := request ()
627- if err == nil {
628- return msgID , nil
629- }
630- if remainTime <= 0 {
631- pc .log .WithError (err ).Error ("Failed to getLastMessageID" )
632- return nil , fmt .Errorf ("failed to getLastMessageID due to %w" , err )
633- }
625+
626+ ctx , cancel := context .WithTimeout (context .Background (), pc .client .operationTimeout )
627+ defer cancel ()
628+ res , err := internal .Retry (ctx , request , func (err error ) time.Duration {
634629 nextDelay := bo .Next ()
635- if nextDelay > remainTime {
636- nextDelay = remainTime
637- }
638- remainTime -= nextDelay
639630 pc .log .WithError (err ).Errorf ("Failed to get last message id from broker, retrying in %v..." , nextDelay )
640- time .Sleep (nextDelay )
631+ return nextDelay
632+ })
633+ if err != nil {
634+ pc .log .WithError (err ).Error ("Failed to getLastMessageID" )
635+ return nil , fmt .Errorf ("failed to getLastMessageID due to %w" , err )
641636 }
637+
638+ return res , nil
642639}
643640
644641func (pc * partitionConsumer ) internalGetLastMessageID (req * getLastMsgIDRequest ) {
@@ -1805,8 +1802,7 @@ func (pc *partitionConsumer) reconnectToBroker(connectionClosed *connectionClose
18051802 pc .log .Debug ("seek operation triggers reconnection, and reset isSeeking" )
18061803 }
18071804 var (
1808- maxRetry int
1809- delayReconnectTime , totalDelayReconnectTime time.Duration
1805+ maxRetry int
18101806 )
18111807
18121808 if pc .options .maxReconnectToBroker == nil {
@@ -1816,50 +1812,39 @@ func (pc *partitionConsumer) reconnectToBroker(connectionClosed *connectionClose
18161812 }
18171813 bo := pc .backoffPolicyFunc ()
18181814
1819- for maxRetry != 0 {
1820- if pc .getConsumerState () != consumerReady {
1821- // Consumer is already closing
1822- pc .log .Info ("consumer state not ready, exit reconnect" )
1823- return
1824- }
1825-
1826- var assignedBrokerURL string
1815+ var assignedBrokerURL string
1816+ if connectionClosed != nil && connectionClosed .HasURL () {
1817+ assignedBrokerURL = connectionClosed .assignedBrokerURL
1818+ }
18271819
1828- if connectionClosed != nil && connectionClosed .HasURL () {
1829- delayReconnectTime = 0
1830- assignedBrokerURL = connectionClosed .assignedBrokerURL
1831- connectionClosed = nil // Attempt connecting to the assigned broker just once
1832- } else {
1833- delayReconnectTime = bo .Next ()
1820+ opFn := func () (struct {}, error ) {
1821+ if maxRetry == 0 {
1822+ return struct {}{}, nil
18341823 }
1835- totalDelayReconnectTime += delayReconnectTime
1836-
1837- pc .log .WithFields (log.Fields {
1838- "assignedBrokerURL" : assignedBrokerURL ,
1839- "delayReconnectTime" : delayReconnectTime ,
1840- }).Info ("Reconnecting to broker" )
1841- time .Sleep (delayReconnectTime )
18421824
1843- // double check
18441825 if pc .getConsumerState () != consumerReady {
18451826 // Consumer is already closing
18461827 pc .log .Info ("consumer state not ready, exit reconnect" )
1847- return
1828+ return struct {}{}, nil
18481829 }
18491830
18501831 err := pc .grabConn (assignedBrokerURL )
1832+ if assignedBrokerURL != "" {
1833+ // Attempt connecting to the assigned broker just once
1834+ assignedBrokerURL = ""
1835+ }
18511836 if err == nil {
18521837 // Successfully reconnected
18531838 pc .log .Info ("Reconnected consumer to broker" )
18541839 bo .Reset ()
1855- return
1840+ return struct {}{}, nil
18561841 }
18571842 pc .log .WithError (err ).Error ("Failed to create consumer at reconnect" )
18581843 errMsg := err .Error ()
18591844 if strings .Contains (errMsg , errMsgTopicNotFound ) {
18601845 // when topic is deleted, we should give up reconnection.
18611846 pc .log .Warn ("Topic Not Found." )
1862- break
1847+ return struct {}{}, nil
18631848 }
18641849
18651850 if maxRetry > 0 {
@@ -1869,7 +1854,17 @@ func (pc *partitionConsumer) reconnectToBroker(connectionClosed *connectionClose
18691854 if maxRetry == 0 || bo .IsMaxBackoffReached () {
18701855 pc .metrics .ConsumersReconnectMaxRetry .Inc ()
18711856 }
1857+
1858+ return struct {}{}, err
18721859 }
1860+ _ , _ = internal .Retry (context .Background (), opFn , func (_ error ) time.Duration {
1861+ delayReconnectTime := bo .Next ()
1862+ pc .log .WithFields (log.Fields {
1863+ "assignedBrokerURL" : assignedBrokerURL ,
1864+ "delayReconnectTime" : delayReconnectTime ,
1865+ }).Info ("Reconnecting to broker" )
1866+ return delayReconnectTime
1867+ })
18731868}
18741869
18751870func (pc * partitionConsumer ) lookupTopic (brokerServiceURL string ) (* internal.LookupResult , error ) {
0 commit comments