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.
		
		
		
		
		
			
		
			
				
					
					
						
							169 lines
						
					
					
						
							5.7 KiB
						
					
					
				
			
		
		
		
			
			
			
		
		
	
	
							169 lines
						
					
					
						
							5.7 KiB
						
					
					
				
								package broker
							 | 
						|
								
							 | 
						|
								import (
							 | 
						|
									"time"
							 | 
						|
								
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/mq/topic"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/util"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
							 | 
						|
									"google.golang.org/protobuf/proto"
							 | 
						|
								)
							 | 
						|
								
							 | 
						|
								// OffsetAssignmentFunc is a function type for assigning offsets to messages
							 | 
						|
								type OffsetAssignmentFunc func() (int64, error)
							 | 
						|
								
							 | 
						|
								// AddToBufferWithOffset adds a message to the log buffer with offset assignment
							 | 
						|
								// TODO: This is a temporary solution until LogBuffer can be modified to accept offset assignment
							 | 
						|
								// ASSUMPTION: This function will be integrated into LogBuffer.AddToBuffer in the future
							 | 
						|
								func (b *MessageQueueBroker) AddToBufferWithOffset(
							 | 
						|
									logBuffer *log_buffer.LogBuffer,
							 | 
						|
									message *mq_pb.DataMessage,
							 | 
						|
									t topic.Topic,
							 | 
						|
									p topic.Partition,
							 | 
						|
								) error {
							 | 
						|
									// Assign offset for this message
							 | 
						|
									offset, err := b.offsetManager.AssignOffset(t, p)
							 | 
						|
									if err != nil {
							 | 
						|
										return err
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// PERFORMANCE OPTIMIZATION: Pre-process expensive operations OUTSIDE the lock
							 | 
						|
									var ts time.Time
							 | 
						|
									processingTsNs := message.TsNs
							 | 
						|
									if processingTsNs == 0 {
							 | 
						|
										ts = time.Now()
							 | 
						|
										processingTsNs = ts.UnixNano()
							 | 
						|
									} else {
							 | 
						|
										ts = time.Unix(0, processingTsNs)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Create LogEntry with assigned offset
							 | 
						|
									logEntry := &filer_pb.LogEntry{
							 | 
						|
										TsNs:             processingTsNs,
							 | 
						|
										PartitionKeyHash: util.HashToInt32(message.Key),
							 | 
						|
										Data:             message.Value,
							 | 
						|
										Key:              message.Key,
							 | 
						|
										Offset:           offset, // Add the assigned offset
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									logEntryData, err := proto.Marshal(logEntry)
							 | 
						|
									if err != nil {
							 | 
						|
										return err
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Use the existing LogBuffer infrastructure for the rest
							 | 
						|
									// TODO: This is a workaround - ideally LogBuffer should handle offset assignment
							 | 
						|
									// For now, we'll add the message with the pre-assigned offset
							 | 
						|
									return b.addLogEntryToBuffer(logBuffer, logEntry, logEntryData, ts)
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								// addLogEntryToBuffer adds a pre-constructed LogEntry to the buffer
							 | 
						|
								// This is a helper function that mimics LogBuffer.AddDataToBuffer but with a pre-built LogEntry
							 | 
						|
								func (b *MessageQueueBroker) addLogEntryToBuffer(
							 | 
						|
									logBuffer *log_buffer.LogBuffer,
							 | 
						|
									logEntry *filer_pb.LogEntry,
							 | 
						|
									logEntryData []byte,
							 | 
						|
									ts time.Time,
							 | 
						|
								) error {
							 | 
						|
									// TODO: This is a simplified version of LogBuffer.AddDataToBuffer
							 | 
						|
									// ASSUMPTION: We're bypassing some of the LogBuffer's internal logic
							 | 
						|
									// This should be properly integrated when LogBuffer is modified
							 | 
						|
								
							 | 
						|
									// Use the new AddLogEntryToBuffer method to preserve offset information
							 | 
						|
									// This ensures the offset is maintained throughout the entire data flow
							 | 
						|
									logBuffer.AddLogEntryToBuffer(logEntry)
							 | 
						|
									return nil
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								// GetPartitionOffsetInfoInternal returns offset information for a partition (internal method)
							 | 
						|
								func (b *MessageQueueBroker) GetPartitionOffsetInfoInternal(t topic.Topic, p topic.Partition) (*PartitionOffsetInfo, error) {
							 | 
						|
									info, err := b.offsetManager.GetPartitionOffsetInfo(t, p)
							 | 
						|
									if err != nil {
							 | 
						|
										return nil, err
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// CRITICAL FIX: Also check LogBuffer for in-memory messages
							 | 
						|
									// The offset manager only tracks assigned offsets from persistent storage
							 | 
						|
									// But the LogBuffer contains recently written messages that haven't been flushed yet
							 | 
						|
									localPartition := b.localTopicManager.GetLocalPartition(t, p)
							 | 
						|
									logBufferHWM := int64(-1)
							 | 
						|
									if localPartition != nil && localPartition.LogBuffer != nil {
							 | 
						|
										logBufferHWM = localPartition.LogBuffer.GetOffset()
							 | 
						|
									} else {
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Use the MAX of offset manager HWM and LogBuffer HWM
							 | 
						|
									// This ensures we report the correct HWM even if data hasn't been flushed to disk yet
							 | 
						|
									// IMPORTANT: Use >= not > because when they're equal, we still want the correct value
							 | 
						|
									highWaterMark := info.HighWaterMark
							 | 
						|
									if logBufferHWM >= 0 && logBufferHWM > highWaterMark {
							 | 
						|
										highWaterMark = logBufferHWM
							 | 
						|
									} else if logBufferHWM >= 0 && logBufferHWM == highWaterMark && highWaterMark > 0 {
							 | 
						|
									} else if logBufferHWM >= 0 {
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Latest offset is HWM - 1 (last assigned offset)
							 | 
						|
									latestOffset := highWaterMark - 1
							 | 
						|
									if highWaterMark == 0 {
							 | 
						|
										latestOffset = -1 // No records
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Convert to broker-specific format
							 | 
						|
									return &PartitionOffsetInfo{
							 | 
						|
										Topic:               t,
							 | 
						|
										Partition:           p,
							 | 
						|
										EarliestOffset:      info.EarliestOffset,
							 | 
						|
										LatestOffset:        latestOffset,
							 | 
						|
										HighWaterMark:       highWaterMark,
							 | 
						|
										RecordCount:         highWaterMark, // HWM equals record count (offsets 0 to HWM-1)
							 | 
						|
										ActiveSubscriptions: info.ActiveSubscriptions,
							 | 
						|
									}, nil
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								// PartitionOffsetInfo provides offset information for a partition (broker-specific)
							 | 
						|
								type PartitionOffsetInfo struct {
							 | 
						|
									Topic               topic.Topic
							 | 
						|
									Partition           topic.Partition
							 | 
						|
									EarliestOffset      int64
							 | 
						|
									LatestOffset        int64
							 | 
						|
									HighWaterMark       int64
							 | 
						|
									RecordCount         int64
							 | 
						|
									ActiveSubscriptions int64
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								// CreateOffsetSubscription creates an offset-based subscription through the broker
							 | 
						|
								func (b *MessageQueueBroker) CreateOffsetSubscription(
							 | 
						|
									subscriptionID string,
							 | 
						|
									t topic.Topic,
							 | 
						|
									p topic.Partition,
							 | 
						|
									offsetType string, // Will be converted to schema_pb.OffsetType
							 | 
						|
									startOffset int64,
							 | 
						|
								) error {
							 | 
						|
									// TODO: Convert string offsetType to schema_pb.OffsetType
							 | 
						|
									// ASSUMPTION: For now using RESET_TO_EARLIEST as default
							 | 
						|
									// This should be properly mapped based on the offsetType parameter
							 | 
						|
								
							 | 
						|
									_, err := b.offsetManager.CreateSubscription(
							 | 
						|
										subscriptionID,
							 | 
						|
										t,
							 | 
						|
										p,
							 | 
						|
										0, // schema_pb.OffsetType_RESET_TO_EARLIEST
							 | 
						|
										startOffset,
							 | 
						|
									)
							 | 
						|
								
							 | 
						|
									return err
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								// GetOffsetMetrics returns offset metrics for monitoring
							 | 
						|
								func (b *MessageQueueBroker) GetOffsetMetrics() map[string]interface{} {
							 | 
						|
									metrics := b.offsetManager.GetOffsetMetrics()
							 | 
						|
								
							 | 
						|
									return map[string]interface{}{
							 | 
						|
										"partition_count":      metrics.PartitionCount,
							 | 
						|
										"total_offsets":        metrics.TotalOffsets,
							 | 
						|
										"active_subscriptions": metrics.ActiveSubscriptions,
							 | 
						|
										"average_latency":      metrics.AverageLatency,
							 | 
						|
									}
							 | 
						|
								}
							 |