You can not select more than 25 topics
			Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
		
		
		
		
		
			
		
			
				
					
					
						
							423 lines
						
					
					
						
							15 KiB
						
					
					
				
			
		
		
		
			
			
			
		
		
	
	
							423 lines
						
					
					
						
							15 KiB
						
					
					
				
								package broker
							 | 
						|
								
							 | 
						|
								import (
							 | 
						|
									"context"
							 | 
						|
									"fmt"
							 | 
						|
									"io"
							 | 
						|
									"time"
							 | 
						|
								
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/glog"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/mq/sub_coordinator"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/mq/topic"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/pb"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
							 | 
						|
								)
							 | 
						|
								
							 | 
						|
								func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_SubscribeMessageServer) error {
							 | 
						|
								
							 | 
						|
									req, err := stream.Recv()
							 | 
						|
									if err != nil {
							 | 
						|
										return err
							 | 
						|
									}
							 | 
						|
									if req.GetInit() == nil {
							 | 
						|
										glog.Errorf("missing init message")
							 | 
						|
										return fmt.Errorf("missing init message")
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Create a cancellable context so we can properly clean up when the client disconnects
							 | 
						|
									ctx, cancel := context.WithCancel(stream.Context())
							 | 
						|
									defer cancel() // Ensure context is cancelled when function exits
							 | 
						|
								
							 | 
						|
									clientName := fmt.Sprintf("%s/%s-%s", req.GetInit().ConsumerGroup, req.GetInit().ConsumerId, req.GetInit().ClientId)
							 | 
						|
								
							 | 
						|
									t := topic.FromPbTopic(req.GetInit().Topic)
							 | 
						|
									partition := topic.FromPbPartition(req.GetInit().GetPartitionOffset().GetPartition())
							 | 
						|
								
							 | 
						|
									glog.V(0).Infof("Subscriber %s on %v %v connected", req.GetInit().ConsumerId, t, partition)
							 | 
						|
								
							 | 
						|
									glog.V(4).Infof("Calling GetOrGenerateLocalPartition for %s %s", t, partition)
							 | 
						|
									localTopicPartition, getOrGenErr := b.GetOrGenerateLocalPartition(t, partition)
							 | 
						|
									if getOrGenErr != nil {
							 | 
						|
										glog.V(4).Infof("GetOrGenerateLocalPartition failed: %v", getOrGenErr)
							 | 
						|
										return getOrGenErr
							 | 
						|
									}
							 | 
						|
									glog.V(4).Infof("GetOrGenerateLocalPartition succeeded, localTopicPartition=%v", localTopicPartition != nil)
							 | 
						|
									if localTopicPartition == nil {
							 | 
						|
										return fmt.Errorf("failed to get or generate local partition for topic %v partition %v", t, partition)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									subscriber := topic.NewLocalSubscriber()
							 | 
						|
									localTopicPartition.Subscribers.AddSubscriber(clientName, subscriber)
							 | 
						|
									glog.V(0).Infof("Subscriber %s connected on %v %v", clientName, t, partition)
							 | 
						|
									isConnected := true
							 | 
						|
								
							 | 
						|
									var counter int64
							 | 
						|
									startPosition := b.getRequestPosition(req.GetInit())
							 | 
						|
									imt := sub_coordinator.NewInflightMessageTracker(int(req.GetInit().SlidingWindowSize))
							 | 
						|
								
							 | 
						|
									defer func() {
							 | 
						|
										isConnected = false
							 | 
						|
										// Clean up any in-flight messages to prevent them from blocking other subscribers
							 | 
						|
										if cleanedCount := imt.Cleanup(); cleanedCount > 0 {
							 | 
						|
											glog.V(0).Infof("Subscriber %s cleaned up %d in-flight messages on disconnect", clientName, cleanedCount)
							 | 
						|
										}
							 | 
						|
										localTopicPartition.Subscribers.RemoveSubscriber(clientName)
							 | 
						|
										glog.V(0).Infof("Subscriber %s on %v %v disconnected, sent %d", clientName, t, partition, counter)
							 | 
						|
										// Use topic-aware shutdown logic to prevent aggressive removal of system topics
							 | 
						|
										if localTopicPartition.MaybeShutdownLocalPartitionForTopic(t.Name) {
							 | 
						|
											b.localTopicManager.RemoveLocalPartition(t, partition)
							 | 
						|
										}
							 | 
						|
									}()
							 | 
						|
								
							 | 
						|
									// connect to the follower
							 | 
						|
									var subscribeFollowMeStream mq_pb.SeaweedMessaging_SubscribeFollowMeClient
							 | 
						|
									glog.V(0).Infof("follower broker: %v", req.GetInit().FollowerBroker)
							 | 
						|
									if req.GetInit().FollowerBroker != "" {
							 | 
						|
										follower := req.GetInit().FollowerBroker
							 | 
						|
										if followerGrpcConnection, err := pb.GrpcDial(ctx, follower, true, b.grpcDialOption); err != nil {
							 | 
						|
											return fmt.Errorf("fail to dial %s: %v", follower, err)
							 | 
						|
										} else {
							 | 
						|
											defer func() {
							 | 
						|
												println("closing SubscribeFollowMe connection", follower)
							 | 
						|
												if subscribeFollowMeStream != nil {
							 | 
						|
													subscribeFollowMeStream.CloseSend()
							 | 
						|
												}
							 | 
						|
												// followerGrpcConnection.Close()
							 | 
						|
											}()
							 | 
						|
											followerClient := mq_pb.NewSeaweedMessagingClient(followerGrpcConnection)
							 | 
						|
											if subscribeFollowMeStream, err = followerClient.SubscribeFollowMe(ctx); err != nil {
							 | 
						|
												return fmt.Errorf("fail to subscribe to %s: %v", follower, err)
							 | 
						|
											} else {
							 | 
						|
												if err := subscribeFollowMeStream.Send(&mq_pb.SubscribeFollowMeRequest{
							 | 
						|
													Message: &mq_pb.SubscribeFollowMeRequest_Init{
							 | 
						|
														Init: &mq_pb.SubscribeFollowMeRequest_InitMessage{
							 | 
						|
															Topic:         req.GetInit().Topic,
							 | 
						|
															Partition:     req.GetInit().GetPartitionOffset().Partition,
							 | 
						|
															ConsumerGroup: req.GetInit().ConsumerGroup,
							 | 
						|
														},
							 | 
						|
													},
							 | 
						|
												}); err != nil {
							 | 
						|
													return fmt.Errorf("fail to send init to %s: %v", follower, err)
							 | 
						|
												}
							 | 
						|
											}
							 | 
						|
										}
							 | 
						|
										glog.V(0).Infof("follower %s connected", follower)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Channel to handle seek requests - signals Subscribe loop to restart from new offset
							 | 
						|
									seekChan := make(chan *mq_pb.SubscribeMessageRequest_SeekMessage, 1)
							 | 
						|
								
							 | 
						|
									go func() {
							 | 
						|
										defer cancel() // CRITICAL: Cancel context when Recv goroutine exits (client disconnect)
							 | 
						|
								
							 | 
						|
										var lastOffset int64
							 | 
						|
								
							 | 
						|
										for {
							 | 
						|
											ack, err := stream.Recv()
							 | 
						|
								
							 | 
						|
											if err != nil {
							 | 
						|
												if err == io.EOF {
							 | 
						|
													// the client has called CloseSend(). This is to ack the close.
							 | 
						|
													stream.Send(&mq_pb.SubscribeMessageResponse{Message: &mq_pb.SubscribeMessageResponse_Ctrl{
							 | 
						|
														Ctrl: &mq_pb.SubscribeMessageResponse_SubscribeCtrlMessage{
							 | 
						|
															IsEndOfStream: true,
							 | 
						|
														},
							 | 
						|
													}})
							 | 
						|
													break
							 | 
						|
												}
							 | 
						|
												glog.V(0).Infof("topic %v partition %v subscriber %s lastOffset %d error: %v", t, partition, clientName, lastOffset, err)
							 | 
						|
												break
							 | 
						|
											}
							 | 
						|
											// Handle seek messages
							 | 
						|
											if seekMsg := ack.GetSeek(); seekMsg != nil {
							 | 
						|
												glog.V(0).Infof("Subscriber %s received seek request to offset %d (type %v)",
							 | 
						|
													clientName, seekMsg.Offset, seekMsg.OffsetType)
							 | 
						|
								
							 | 
						|
												// Send seek request to Subscribe loop
							 | 
						|
												select {
							 | 
						|
												case seekChan <- seekMsg:
							 | 
						|
													glog.V(0).Infof("Subscriber %s seek request queued", clientName)
							 | 
						|
												default:
							 | 
						|
													glog.V(0).Infof("Subscriber %s seek request dropped (already pending)", clientName)
							 | 
						|
													// Send error response if seek is already in progress
							 | 
						|
													stream.Send(&mq_pb.SubscribeMessageResponse{Message: &mq_pb.SubscribeMessageResponse_Ctrl{
							 | 
						|
														Ctrl: &mq_pb.SubscribeMessageResponse_SubscribeCtrlMessage{
							 | 
						|
															Error: "Seek already in progress",
							 | 
						|
														},
							 | 
						|
													}})
							 | 
						|
												}
							 | 
						|
												continue
							 | 
						|
											}
							 | 
						|
								
							 | 
						|
											if ack.GetAck().Key == nil {
							 | 
						|
												// skip ack for control messages
							 | 
						|
												continue
							 | 
						|
											}
							 | 
						|
											imt.AcknowledgeMessage(ack.GetAck().Key, ack.GetAck().TsNs)
							 | 
						|
								
							 | 
						|
											currentLastOffset := imt.GetOldestAckedTimestamp()
							 | 
						|
											// Update acknowledged offset and last seen time for this subscriber when it sends an ack
							 | 
						|
											subscriber.UpdateAckedOffset(currentLastOffset)
							 | 
						|
											// fmt.Printf("%+v recv (%s,%d), oldest %d\n", partition, string(ack.GetAck().Key), ack.GetAck().TsNs, currentLastOffset)
							 | 
						|
											if subscribeFollowMeStream != nil && currentLastOffset > lastOffset {
							 | 
						|
												if err := subscribeFollowMeStream.Send(&mq_pb.SubscribeFollowMeRequest{
							 | 
						|
													Message: &mq_pb.SubscribeFollowMeRequest_Ack{
							 | 
						|
														Ack: &mq_pb.SubscribeFollowMeRequest_AckMessage{
							 | 
						|
															TsNs: currentLastOffset,
							 | 
						|
														},
							 | 
						|
													},
							 | 
						|
												}); err != nil {
							 | 
						|
													glog.Errorf("Error sending ack to follower: %v", err)
							 | 
						|
													break
							 | 
						|
												}
							 | 
						|
												lastOffset = currentLastOffset
							 | 
						|
												// fmt.Printf("%+v forwarding ack %d\n", partition, lastOffset)
							 | 
						|
											}
							 | 
						|
										}
							 | 
						|
										if lastOffset > 0 {
							 | 
						|
											glog.V(0).Infof("saveConsumerGroupOffset %v %v %v %v", t, partition, req.GetInit().ConsumerGroup, lastOffset)
							 | 
						|
											if err := b.saveConsumerGroupOffset(t, partition, req.GetInit().ConsumerGroup, lastOffset); err != nil {
							 | 
						|
												glog.Errorf("saveConsumerGroupOffset partition %v lastOffset %d: %v", partition, lastOffset, err)
							 | 
						|
											}
							 | 
						|
										}
							 | 
						|
										if subscribeFollowMeStream != nil {
							 | 
						|
											if err := subscribeFollowMeStream.Send(&mq_pb.SubscribeFollowMeRequest{
							 | 
						|
												Message: &mq_pb.SubscribeFollowMeRequest_Close{
							 | 
						|
													Close: &mq_pb.SubscribeFollowMeRequest_CloseMessage{},
							 | 
						|
												},
							 | 
						|
											}); err != nil {
							 | 
						|
												if err != io.EOF {
							 | 
						|
													glog.Errorf("Error sending close to follower: %v", err)
							 | 
						|
												}
							 | 
						|
											}
							 | 
						|
										}
							 | 
						|
									}()
							 | 
						|
								
							 | 
						|
									// Create a goroutine to handle context cancellation and wake up the condition variable
							 | 
						|
									// This is created ONCE per subscriber, not per callback invocation
							 | 
						|
									go func() {
							 | 
						|
										<-ctx.Done()
							 | 
						|
										// Wake up the condition variable when context is cancelled
							 | 
						|
										localTopicPartition.ListenersLock.Lock()
							 | 
						|
										localTopicPartition.ListenersCond.Broadcast()
							 | 
						|
										localTopicPartition.ListenersLock.Unlock()
							 | 
						|
									}()
							 | 
						|
								
							 | 
						|
									// Subscribe loop - can be restarted when seek is requested
							 | 
						|
									currentPosition := startPosition
							 | 
						|
								subscribeLoop:
							 | 
						|
									for {
							 | 
						|
										// Context for this iteration of Subscribe (can be cancelled by seek)
							 | 
						|
										subscribeCtx, subscribeCancel := context.WithCancel(ctx)
							 | 
						|
								
							 | 
						|
										// Start Subscribe in a goroutine so we can interrupt it with seek
							 | 
						|
										subscribeDone := make(chan error, 1)
							 | 
						|
										go func() {
							 | 
						|
											subscribeErr := localTopicPartition.Subscribe(clientName, currentPosition, func() bool {
							 | 
						|
												// Check cancellation before waiting
							 | 
						|
												if subscribeCtx.Err() != nil || !isConnected {
							 | 
						|
													return false
							 | 
						|
												}
							 | 
						|
								
							 | 
						|
												// Wait for new data using condition variable (blocking, not polling)
							 | 
						|
												localTopicPartition.ListenersLock.Lock()
							 | 
						|
												localTopicPartition.ListenersCond.Wait()
							 | 
						|
												localTopicPartition.ListenersLock.Unlock()
							 | 
						|
								
							 | 
						|
												// After waking up, check if we should stop
							 | 
						|
												return subscribeCtx.Err() == nil && isConnected
							 | 
						|
											}, func(logEntry *filer_pb.LogEntry) (bool, error) {
							 | 
						|
												// Wait for the message to be acknowledged with a timeout to prevent infinite loops
							 | 
						|
												const maxWaitTime = 30 * time.Second
							 | 
						|
												const checkInterval = 137 * time.Millisecond
							 | 
						|
												startTime := time.Now()
							 | 
						|
								
							 | 
						|
												for imt.IsInflight(logEntry.Key) {
							 | 
						|
													// Check if we've exceeded the maximum wait time
							 | 
						|
													if time.Since(startTime) > maxWaitTime {
							 | 
						|
														glog.Warningf("Subscriber %s: message with key %s has been in-flight for more than %v, forcing acknowledgment",
							 | 
						|
															clientName, string(logEntry.Key), maxWaitTime)
							 | 
						|
														// Force remove the message from in-flight tracking to prevent infinite loop
							 | 
						|
														imt.AcknowledgeMessage(logEntry.Key, logEntry.TsNs)
							 | 
						|
														break
							 | 
						|
													}
							 | 
						|
								
							 | 
						|
													time.Sleep(checkInterval)
							 | 
						|
								
							 | 
						|
													// Check if the client has disconnected by monitoring the context
							 | 
						|
													select {
							 | 
						|
													case <-subscribeCtx.Done():
							 | 
						|
														err := subscribeCtx.Err()
							 | 
						|
														if err == context.Canceled {
							 | 
						|
															// Subscribe cancelled (seek or disconnect)
							 | 
						|
															return false, nil
							 | 
						|
														}
							 | 
						|
														glog.V(0).Infof("Subscriber %s disconnected: %v", clientName, err)
							 | 
						|
														return false, nil
							 | 
						|
													default:
							 | 
						|
														// Continue processing the request
							 | 
						|
													}
							 | 
						|
												}
							 | 
						|
												if logEntry.Key != nil {
							 | 
						|
													imt.EnflightMessage(logEntry.Key, logEntry.TsNs)
							 | 
						|
												}
							 | 
						|
								
							 | 
						|
												// Create the message to send
							 | 
						|
												dataMsg := &mq_pb.DataMessage{
							 | 
						|
													Key:   logEntry.Key,
							 | 
						|
													Value: logEntry.Data,
							 | 
						|
													TsNs:  logEntry.TsNs,
							 | 
						|
												}
							 | 
						|
								
							 | 
						|
								
							 | 
						|
												if err := stream.Send(&mq_pb.SubscribeMessageResponse{Message: &mq_pb.SubscribeMessageResponse_Data{
							 | 
						|
													Data: dataMsg,
							 | 
						|
												}}); err != nil {
							 | 
						|
													glog.Errorf("Error sending data: %v", err)
							 | 
						|
													return false, err
							 | 
						|
												}
							 | 
						|
								
							 | 
						|
												// Update received offset and last seen time for this subscriber
							 | 
						|
												subscriber.UpdateReceivedOffset(logEntry.TsNs)
							 | 
						|
								
							 | 
						|
												counter++
							 | 
						|
												return false, nil
							 | 
						|
											})
							 | 
						|
											subscribeDone <- subscribeErr
							 | 
						|
										}()
							 | 
						|
								
							 | 
						|
										// Wait for either Subscribe to complete or a seek request
							 | 
						|
										select {
							 | 
						|
										case err = <-subscribeDone:
							 | 
						|
											subscribeCancel()
							 | 
						|
											if err != nil || ctx.Err() != nil {
							 | 
						|
												// Subscribe finished with error or main context cancelled - exit loop
							 | 
						|
												break subscribeLoop
							 | 
						|
											}
							 | 
						|
											// Subscribe completed normally (shouldn't happen in streaming mode)
							 | 
						|
											break subscribeLoop
							 | 
						|
								
							 | 
						|
										case seekMsg := <-seekChan:
							 | 
						|
											// Seek requested - cancel current Subscribe and restart from new offset
							 | 
						|
											glog.V(0).Infof("Subscriber %s seeking from offset %d to offset %d (type %v)",
							 | 
						|
												clientName, currentPosition.GetOffset(), seekMsg.Offset, seekMsg.OffsetType)
							 | 
						|
								
							 | 
						|
											// Cancel current Subscribe iteration
							 | 
						|
											subscribeCancel()
							 | 
						|
								
							 | 
						|
											// Wait for Subscribe to finish cancelling
							 | 
						|
											<-subscribeDone
							 | 
						|
								
							 | 
						|
											// Update position for next iteration
							 | 
						|
											currentPosition = b.getRequestPositionFromSeek(seekMsg)
							 | 
						|
											glog.V(0).Infof("Subscriber %s restarting Subscribe from new offset %d", clientName, seekMsg.Offset)
							 | 
						|
								
							 | 
						|
											// Send acknowledgment that seek completed
							 | 
						|
											stream.Send(&mq_pb.SubscribeMessageResponse{Message: &mq_pb.SubscribeMessageResponse_Ctrl{
							 | 
						|
												Ctrl: &mq_pb.SubscribeMessageResponse_SubscribeCtrlMessage{
							 | 
						|
													Error: "", // Empty error means success
							 | 
						|
												},
							 | 
						|
											}})
							 | 
						|
								
							 | 
						|
											// Loop will restart with new position
							 | 
						|
										}
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									return err
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								func (b *MessageQueueBroker) getRequestPosition(initMessage *mq_pb.SubscribeMessageRequest_InitMessage) (startPosition log_buffer.MessagePosition) {
							 | 
						|
									if initMessage == nil {
							 | 
						|
										return
							 | 
						|
									}
							 | 
						|
									offset := initMessage.GetPartitionOffset()
							 | 
						|
									offsetType := initMessage.OffsetType
							 | 
						|
								
							 | 
						|
									// reset to earliest or latest
							 | 
						|
									if offsetType == schema_pb.OffsetType_RESET_TO_EARLIEST {
							 | 
						|
										startPosition = log_buffer.NewMessagePosition(1, -3)
							 | 
						|
										return
							 | 
						|
									}
							 | 
						|
									if offsetType == schema_pb.OffsetType_RESET_TO_LATEST {
							 | 
						|
										startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -4)
							 | 
						|
										return
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// use the exact timestamp
							 | 
						|
									if offsetType == schema_pb.OffsetType_EXACT_TS_NS {
							 | 
						|
										startPosition = log_buffer.NewMessagePosition(offset.StartTsNs, -2)
							 | 
						|
										return
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// use exact offset (native offset-based positioning)
							 | 
						|
									if offsetType == schema_pb.OffsetType_EXACT_OFFSET {
							 | 
						|
										startPosition = log_buffer.NewMessagePositionFromOffset(offset.StartOffset)
							 | 
						|
										return
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// reset to specific offset
							 | 
						|
									if offsetType == schema_pb.OffsetType_RESET_TO_OFFSET {
							 | 
						|
										startPosition = log_buffer.NewMessagePositionFromOffset(offset.StartOffset)
							 | 
						|
										return
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// try to resume
							 | 
						|
									if storedOffset, err := b.readConsumerGroupOffset(initMessage); err == nil {
							 | 
						|
										glog.V(0).Infof("resume from saved offset %v %v %v: %v", initMessage.Topic, initMessage.PartitionOffset.Partition, initMessage.ConsumerGroup, storedOffset)
							 | 
						|
										startPosition = log_buffer.NewMessagePosition(storedOffset, -2)
							 | 
						|
										return
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									if offsetType == schema_pb.OffsetType_RESUME_OR_EARLIEST {
							 | 
						|
										startPosition = log_buffer.NewMessagePosition(1, -5)
							 | 
						|
									} else if offsetType == schema_pb.OffsetType_RESUME_OR_LATEST {
							 | 
						|
										startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -6)
							 | 
						|
									}
							 | 
						|
									return
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								// getRequestPositionFromSeek converts a seek request to a MessagePosition
							 | 
						|
								// This is used when implementing full seek support in Subscribe loop
							 | 
						|
								func (b *MessageQueueBroker) getRequestPositionFromSeek(seekMsg *mq_pb.SubscribeMessageRequest_SeekMessage) (startPosition log_buffer.MessagePosition) {
							 | 
						|
									if seekMsg == nil {
							 | 
						|
										return
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									offsetType := seekMsg.OffsetType
							 | 
						|
									offset := seekMsg.Offset
							 | 
						|
								
							 | 
						|
									// reset to earliest or latest
							 | 
						|
									if offsetType == schema_pb.OffsetType_RESET_TO_EARLIEST {
							 | 
						|
										startPosition = log_buffer.NewMessagePosition(1, -3)
							 | 
						|
										return
							 | 
						|
									}
							 | 
						|
									if offsetType == schema_pb.OffsetType_RESET_TO_LATEST {
							 | 
						|
										startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -4)
							 | 
						|
										return
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// use the exact timestamp
							 | 
						|
									if offsetType == schema_pb.OffsetType_EXACT_TS_NS {
							 | 
						|
										startPosition = log_buffer.NewMessagePosition(offset, -2)
							 | 
						|
										return
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// use exact offset (native offset-based positioning)
							 | 
						|
									if offsetType == schema_pb.OffsetType_EXACT_OFFSET {
							 | 
						|
										startPosition = log_buffer.NewMessagePositionFromOffset(offset)
							 | 
						|
										return
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// reset to specific offset
							 | 
						|
									if offsetType == schema_pb.OffsetType_RESET_TO_OFFSET {
							 | 
						|
										startPosition = log_buffer.NewMessagePositionFromOffset(offset)
							 | 
						|
										return
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// default to exact offset
							 | 
						|
									startPosition = log_buffer.NewMessagePositionFromOffset(offset)
							 | 
						|
									return
							 | 
						|
								}
							 |