@@ -190,9 +190,13 @@ type partitionConsumer struct {
190190 backoffPolicyFunc func () backoff.Policy
191191
192192 dispatcherSeekingControlCh chan struct {}
193- isSeeking atomic.Bool
194- ctx context.Context
195- cancelFunc context.CancelFunc
193+ // handle to the dispatcher goroutine
194+ isSeeking atomic.Bool
195+ // After executing seekByTime, the client is unaware of the startMessageId.
196+ // Use this flag to compare markDeletePosition with BrokerLastMessageId when checking hasMoreMessages.
197+ hasSoughtByTime atomic.Bool
198+ ctx context.Context
199+ cancelFunc context.CancelFunc
196200}
197201
198202// pauseDispatchMessage used to discard the message in the dispatcher goroutine.
@@ -429,11 +433,12 @@ func newPartitionConsumer(parent Consumer, client *client, options *partitionCon
429433
430434 startingMessageID := pc .startMessageID .get ()
431435 if pc .options .startMessageIDInclusive && startingMessageID != nil && startingMessageID .equal (latestMessageID ) {
432- msgID , err := pc .requestGetLastMessageID ()
436+ msgIDResp , err := pc .requestGetLastMessageID ()
433437 if err != nil {
434438 pc .Close ()
435439 return nil , err
436440 }
441+ msgID := convertToMessageID (msgIDResp .GetLastMessageId ())
437442 if msgID .entryID != noMessageEntry {
438443 pc .startMessageID .set (msgID )
439444
@@ -616,18 +621,24 @@ func (pc *partitionConsumer) internalUnsubscribe(unsub *unsubscribeRequest) {
616621}
617622
618623func (pc * partitionConsumer ) getLastMessageID () (* trackingMessageID , error ) {
624+ res , err := pc .getLastMessageIDAndMarkDeletePosition ()
625+ return res .msgID , err
626+ }
627+
628+ func (pc * partitionConsumer ) getLastMessageIDAndMarkDeletePosition () (* getLastMsgResult , error ) {
619629 if state := pc .getConsumerState (); state == consumerClosed || state == consumerClosing {
620630 pc .log .WithField ("state" , state ).Error ("Failed to getLastMessageID for the closing or closed consumer" )
621631 return nil , errors .New ("failed to getLastMessageID for the closing or closed consumer" )
622632 }
623633 bo := pc .backoffPolicyFunc ()
624- request := func () (* trackingMessageID , error ) {
634+ request := func () (* getLastMsgResult , error ) {
625635 req := & getLastMsgIDRequest {doneCh : make (chan struct {})}
626636 pc .eventsCh <- req
627637
628638 // wait for the request to complete
629639 <- req .doneCh
630- return req .msgID , req .err
640+ res := & getLastMsgResult {req .msgID , req .markDeletePosition }
641+ return res , req .err
631642 }
632643
633644 ctx , cancel := context .WithTimeout (context .Background (), pc .client .operationTimeout )
@@ -647,10 +658,16 @@ func (pc *partitionConsumer) getLastMessageID() (*trackingMessageID, error) {
647658
648659func (pc * partitionConsumer ) internalGetLastMessageID (req * getLastMsgIDRequest ) {
649660 defer close (req .doneCh )
650- req .msgID , req .err = pc .requestGetLastMessageID ()
661+ rsp , err := pc .requestGetLastMessageID ()
662+ if err != nil {
663+ req .err = err
664+ return
665+ }
666+ req .msgID = convertToMessageID (rsp .GetLastMessageId ())
667+ req .markDeletePosition = convertToMessageID (rsp .GetConsumerMarkDeletePosition ())
651668}
652669
653- func (pc * partitionConsumer ) requestGetLastMessageID () (* trackingMessageID , error ) {
670+ func (pc * partitionConsumer ) requestGetLastMessageID () (* pb. CommandGetLastMessageIdResponse , error ) {
654671 if state := pc .getConsumerState (); state == consumerClosed || state == consumerClosing {
655672 pc .log .WithField ("state" , state ).Error ("Failed to getLastMessageID closing or closed consumer" )
656673 return nil , errors .New ("failed to getLastMessageID closing or closed consumer" )
@@ -667,8 +684,7 @@ func (pc *partitionConsumer) requestGetLastMessageID() (*trackingMessageID, erro
667684 pc .log .WithError (err ).Error ("Failed to get last message id" )
668685 return nil , err
669686 }
670- id := res .Response .GetLastMessageIdResponse .GetLastMessageId ()
671- return convertToMessageID (id ), nil
687+ return res .Response .GetLastMessageIdResponse , nil
672688}
673689
674690func (pc * partitionConsumer ) sendIndividualAck (msgID MessageID ) * ackRequest {
@@ -997,7 +1013,14 @@ func (pc *partitionConsumer) requestSeek(msgID *messageID) error {
9971013 if err := pc .requestSeekWithoutClear (msgID ); err != nil {
9981014 return err
9991015 }
1000- pc .clearReceiverQueue ()
1016+ // When the seek operation is successful, it indicates:
1017+ // 1. The broker has reset the cursor and sent a request to close the consumer on the client side.
1018+ // Since this method is in the same goroutine as the reconnectToBroker,
1019+ // we can safely clear the messages in the queue (at this point, it won't contain messages after the seek).
1020+ // 2. The startMessageID is reset to ensure accurate judgment when calling hasNext next time.
1021+ // Since the messages in the queue are cleared here reconnection won't reset startMessageId.
1022+ pc .lastDequeuedMsg = nil
1023+ pc .startMessageID .set (toTrackingMessageID (msgID ))
10011024 return nil
10021025}
10031026
@@ -1069,7 +1092,9 @@ func (pc *partitionConsumer) internalSeekByTime(seek *seekByTimeRequest) {
10691092 seek .err = err
10701093 return
10711094 }
1072- pc .clearReceiverQueue ()
1095+ pc .clearQueueAndGetNextMessage ()
1096+ pc .lastDequeuedMsg = nil
1097+ pc .hasSoughtByTime .Store (true )
10731098}
10741099
10751100func (pc * partitionConsumer ) internalAck (req * ackRequest ) {
@@ -1451,10 +1476,6 @@ func (pc *partitionConsumer) messageShouldBeDiscarded(msgID *trackingMessageID)
14511476 if pc .startMessageID .get () == nil {
14521477 return false
14531478 }
1454- // if we start at latest message, we should never discard
1455- if pc .options .startMessageID != nil && pc .options .startMessageID .equal (latestMessageID ) {
1456- return false
1457- }
14581479
14591480 if pc .options .startMessageIDInclusive {
14601481 return pc .startMessageID .get ().greater (msgID .messageID )
@@ -1709,9 +1730,15 @@ type redeliveryRequest struct {
17091730}
17101731
17111732type getLastMsgIDRequest struct {
1712- doneCh chan struct {}
1713- msgID * trackingMessageID
1714- err error
1733+ doneCh chan struct {}
1734+ msgID * trackingMessageID
1735+ markDeletePosition * trackingMessageID
1736+ err error
1737+ }
1738+
1739+ type getLastMsgResult struct {
1740+ msgID * trackingMessageID
1741+ markDeletePosition * trackingMessageID
17151742}
17161743
17171744type seekRequest struct {
@@ -2195,6 +2222,24 @@ func (pc *partitionConsumer) discardCorruptedMessage(msgID *pb.MessageIdData,
21952222}
21962223
21972224func (pc * partitionConsumer ) hasNext () bool {
2225+
2226+ // If a seek by time has been performed, then the `startMessageId` becomes irrelevant.
2227+ // We need to compare `markDeletePosition` and `lastMessageId`,
2228+ // and then reset `startMessageID` to `markDeletePosition`.
2229+ if pc .hasSoughtByTime .CompareAndSwap (true , false ) {
2230+ res , err := pc .getLastMessageIDAndMarkDeletePosition ()
2231+ if err != nil {
2232+ pc .log .WithError (err ).Error ("Failed to get last message id" )
2233+ return false
2234+ }
2235+ pc .lastMessageInBroker = res .msgID
2236+ pc .startMessageID .set (res .markDeletePosition )
2237+ // We only care about comparing ledger ids and entry ids as mark delete position
2238+ // doesn't have other ids such as batch index
2239+ compareResult := pc .lastMessageInBroker .messageID .compareLedgerAndEntryId (pc .startMessageID .get ().messageID )
2240+ return compareResult > 0 || (pc .options .startMessageIDInclusive && compareResult == 0 )
2241+ }
2242+
21982243 if pc .lastMessageInBroker != nil && pc .hasMoreMessages () {
21992244 return true
22002245 }
0 commit comments