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.
		
		
		
		
		
			
		
			
				
					
					
						
							707 lines
						
					
					
						
							21 KiB
						
					
					
				
			
		
		
		
			
			
			
		
		
	
	
							707 lines
						
					
					
						
							21 KiB
						
					
					
				
								package broker
							 | 
						|
								
							 | 
						|
								import (
							 | 
						|
									"fmt"
							 | 
						|
									"testing"
							 | 
						|
									"time"
							 | 
						|
								
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/mq/offset"
							 | 
						|
									"github.com/seaweedfs/seaweedfs/weed/mq/topic"
							 | 
						|
									"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 TestConvertOffsetToMessagePosition(t *testing.T) {
							 | 
						|
									broker := &MessageQueueBroker{}
							 | 
						|
								
							 | 
						|
									tests := []struct {
							 | 
						|
										name          string
							 | 
						|
										offsetType    schema_pb.OffsetType
							 | 
						|
										currentOffset int64
							 | 
						|
										expectedBatch int64
							 | 
						|
										expectError   bool
							 | 
						|
									}{
							 | 
						|
										{
							 | 
						|
											name:          "reset to earliest",
							 | 
						|
											offsetType:    schema_pb.OffsetType_RESET_TO_EARLIEST,
							 | 
						|
											currentOffset: 0,
							 | 
						|
											expectedBatch: -3,
							 | 
						|
											expectError:   false,
							 | 
						|
										},
							 | 
						|
										{
							 | 
						|
											name:          "reset to latest",
							 | 
						|
											offsetType:    schema_pb.OffsetType_RESET_TO_LATEST,
							 | 
						|
											currentOffset: 0,
							 | 
						|
											expectedBatch: -4,
							 | 
						|
											expectError:   false,
							 | 
						|
										},
							 | 
						|
										{
							 | 
						|
											name:          "exact offset zero",
							 | 
						|
											offsetType:    schema_pb.OffsetType_EXACT_OFFSET,
							 | 
						|
											currentOffset: 0,
							 | 
						|
											expectedBatch: 0, // NewMessagePositionFromOffset stores offset directly in Offset field
							 | 
						|
											expectError:   false,
							 | 
						|
										},
							 | 
						|
										{
							 | 
						|
											name:          "exact offset non-zero",
							 | 
						|
											offsetType:    schema_pb.OffsetType_EXACT_OFFSET,
							 | 
						|
											currentOffset: 100,
							 | 
						|
											expectedBatch: 100, // NewMessagePositionFromOffset stores offset directly in Offset field
							 | 
						|
											expectError:   false,
							 | 
						|
										},
							 | 
						|
										{
							 | 
						|
											name:          "exact timestamp",
							 | 
						|
											offsetType:    schema_pb.OffsetType_EXACT_TS_NS,
							 | 
						|
											currentOffset: 50,
							 | 
						|
											expectedBatch: -2,
							 | 
						|
											expectError:   false,
							 | 
						|
										},
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									for _, tt := range tests {
							 | 
						|
										t.Run(tt.name, func(t *testing.T) {
							 | 
						|
											// Create a mock subscription
							 | 
						|
											subscription := &offset.OffsetSubscription{
							 | 
						|
												ID:            "test-subscription",
							 | 
						|
												CurrentOffset: tt.currentOffset,
							 | 
						|
												OffsetType:    tt.offsetType,
							 | 
						|
												IsActive:      true,
							 | 
						|
											}
							 | 
						|
								
							 | 
						|
											position, err := broker.convertOffsetToMessagePosition(subscription)
							 | 
						|
								
							 | 
						|
											if tt.expectError && err == nil {
							 | 
						|
												t.Error("Expected error but got none")
							 | 
						|
												return
							 | 
						|
											}
							 | 
						|
								
							 | 
						|
											if !tt.expectError && err != nil {
							 | 
						|
												t.Errorf("Unexpected error: %v", err)
							 | 
						|
												return
							 | 
						|
											}
							 | 
						|
								
							 | 
						|
											if position.Offset != tt.expectedBatch {
							 | 
						|
												t.Errorf("Expected batch index %d, got %d", tt.expectedBatch, position.Offset)
							 | 
						|
											}
							 | 
						|
								
							 | 
						|
											// Verify that the timestamp is reasonable (not zero for most cases)
							 | 
						|
											// Note: EXACT_OFFSET uses epoch time (zero) with NewMessagePositionFromOffset
							 | 
						|
											if tt.offsetType != schema_pb.OffsetType_RESET_TO_EARLIEST &&
							 | 
						|
												tt.offsetType != schema_pb.OffsetType_EXACT_OFFSET &&
							 | 
						|
												position.Time.IsZero() {
							 | 
						|
												t.Error("Expected non-zero timestamp")
							 | 
						|
											}
							 | 
						|
								
							 | 
						|
										})
							 | 
						|
									}
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								func TestConvertOffsetToMessagePosition_OffsetEncoding(t *testing.T) {
							 | 
						|
									broker := &MessageQueueBroker{}
							 | 
						|
								
							 | 
						|
									// Test that offset-based positions encode the offset correctly in Offset field
							 | 
						|
									testCases := []struct {
							 | 
						|
										offset             int64
							 | 
						|
										expectedBatch      int64
							 | 
						|
										expectedIsSentinel bool // Should timestamp be the offset sentinel value?
							 | 
						|
									}{
							 | 
						|
										{10, 10, true},
							 | 
						|
										{100, 100, true},
							 | 
						|
										{0, 0, true},
							 | 
						|
										{42, 42, true},
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									for _, tc := range testCases {
							 | 
						|
										t.Run(fmt.Sprintf("offset_%d", tc.offset), func(t *testing.T) {
							 | 
						|
											subscription := &offset.OffsetSubscription{
							 | 
						|
												ID:            fmt.Sprintf("test-%d", tc.offset),
							 | 
						|
												CurrentOffset: tc.offset,
							 | 
						|
												OffsetType:    schema_pb.OffsetType_EXACT_OFFSET,
							 | 
						|
												IsActive:      true,
							 | 
						|
											}
							 | 
						|
								
							 | 
						|
											pos, err := broker.convertOffsetToMessagePosition(subscription)
							 | 
						|
											if err != nil {
							 | 
						|
												t.Fatalf("Unexpected error: %v", err)
							 | 
						|
											}
							 | 
						|
								
							 | 
						|
											// Check Offset encoding
							 | 
						|
											if pos.Offset != tc.expectedBatch {
							 | 
						|
												t.Errorf("Expected batch index %d, got %d", tc.expectedBatch, pos.Offset)
							 | 
						|
											}
							 | 
						|
								
							 | 
						|
											// Verify the offset can be extracted correctly using IsOffsetBased/GetOffset
							 | 
						|
											if !pos.IsOffsetBased {
							 | 
						|
												t.Error("Position should be detected as offset-based")
							 | 
						|
											}
							 | 
						|
								
							 | 
						|
											// Check that IsOffsetBased flag is set correctly
							 | 
						|
											if tc.expectedIsSentinel && !pos.IsOffsetBased {
							 | 
						|
												t.Error("Expected offset-based position but IsOffsetBased=false")
							 | 
						|
											}
							 | 
						|
								
							 | 
						|
											if extractedOffset := pos.GetOffset(); extractedOffset != tc.offset {
							 | 
						|
												t.Errorf("Expected extracted offset %d, got %d", tc.offset, extractedOffset)
							 | 
						|
											}
							 | 
						|
								
							 | 
						|
										})
							 | 
						|
									}
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								func TestConvertOffsetToMessagePosition_ConsistentResults(t *testing.T) {
							 | 
						|
									broker := &MessageQueueBroker{}
							 | 
						|
								
							 | 
						|
									subscription := &offset.OffsetSubscription{
							 | 
						|
										ID:            "consistent-test",
							 | 
						|
										CurrentOffset: 42,
							 | 
						|
										OffsetType:    schema_pb.OffsetType_EXACT_OFFSET,
							 | 
						|
										IsActive:      true,
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Call multiple times within a short period
							 | 
						|
									positions := make([]log_buffer.MessagePosition, 5)
							 | 
						|
									for i := 0; i < 5; i++ {
							 | 
						|
										pos, err := broker.convertOffsetToMessagePosition(subscription)
							 | 
						|
										if err != nil {
							 | 
						|
											t.Fatalf("Unexpected error on iteration %d: %v", i, err)
							 | 
						|
										}
							 | 
						|
										positions[i] = pos
							 | 
						|
										time.Sleep(1 * time.Millisecond) // Small delay
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// All positions should have the same Offset
							 | 
						|
									for i := 1; i < len(positions); i++ {
							 | 
						|
										if positions[i].Offset != positions[0].Offset {
							 | 
						|
											t.Errorf("Inconsistent Offset: %d vs %d", positions[0].Offset, positions[i].Offset)
							 | 
						|
										}
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// With NewMessagePositionFromOffset, timestamps should be identical (zero time for offset-based)
							 | 
						|
									expectedTime := time.Time{}
							 | 
						|
									for i := 0; i < len(positions); i++ {
							 | 
						|
										if !positions[i].Time.Equal(expectedTime) {
							 | 
						|
											t.Errorf("Expected all timestamps to be sentinel time (%v), got %v at index %d",
							 | 
						|
												expectedTime, positions[i].Time, i)
							 | 
						|
										}
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								func TestConvertOffsetToMessagePosition_FixVerification(t *testing.T) {
							 | 
						|
									// This test specifically verifies that the fix addresses the issue mentioned:
							 | 
						|
									// "The calculated timestamp for a given offset will change every time the function is called"
							 | 
						|
								
							 | 
						|
									broker := &MessageQueueBroker{}
							 | 
						|
								
							 | 
						|
									subscription := &offset.OffsetSubscription{
							 | 
						|
										ID:            "fix-verification",
							 | 
						|
										CurrentOffset: 123,
							 | 
						|
										OffsetType:    schema_pb.OffsetType_EXACT_OFFSET,
							 | 
						|
										IsActive:      true,
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Call the function multiple times with delays to simulate real-world usage
							 | 
						|
									var positions []log_buffer.MessagePosition
							 | 
						|
									var timestamps []int64
							 | 
						|
								
							 | 
						|
									for i := 0; i < 10; i++ {
							 | 
						|
										pos, err := broker.convertOffsetToMessagePosition(subscription)
							 | 
						|
										if err != nil {
							 | 
						|
											t.Fatalf("Unexpected error on iteration %d: %v", i, err)
							 | 
						|
										}
							 | 
						|
										positions = append(positions, pos)
							 | 
						|
										timestamps = append(timestamps, pos.Time.UnixNano())
							 | 
						|
										time.Sleep(2 * time.Millisecond) // Small delay to ensure time progression
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Verify ALL timestamps are identical (no time-based variance)
							 | 
						|
									expectedTimestamp := timestamps[0]
							 | 
						|
									for i, ts := range timestamps {
							 | 
						|
										if ts != expectedTimestamp {
							 | 
						|
											t.Errorf("Timestamp variance detected at call %d: expected %d, got %d", i, expectedTimestamp, ts)
							 | 
						|
										}
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Verify ALL Offset values are identical
							 | 
						|
									expectedBatch := positions[0].Offset
							 | 
						|
									for i, pos := range positions {
							 | 
						|
										if pos.Offset != expectedBatch {
							 | 
						|
											t.Errorf("Offset variance detected at call %d: expected %d, got %d", i, expectedBatch, pos.Offset)
							 | 
						|
										}
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Verify the offset can be consistently extracted
							 | 
						|
									expectedOffset := subscription.CurrentOffset
							 | 
						|
									for i, pos := range positions {
							 | 
						|
										if extractedOffset := pos.GetOffset(); extractedOffset != expectedOffset {
							 | 
						|
											t.Errorf("Extracted offset variance at call %d: expected %d, got %d", i, expectedOffset, extractedOffset)
							 | 
						|
										}
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								func TestPartitionIdentityConsistency(t *testing.T) {
							 | 
						|
									// Test that partition identity is preserved from request to avoid breaking offset manager keys
							 | 
						|
								
							 | 
						|
									// Create a mock init message with specific partition info
							 | 
						|
									partition := &schema_pb.Partition{
							 | 
						|
										RingSize:   32,
							 | 
						|
										RangeStart: 0,
							 | 
						|
										RangeStop:  31,
							 | 
						|
										UnixTimeNs: 1234567890123456789, // Fixed timestamp
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									initMessage := &mq_pb.SubscribeMessageRequest_InitMessage{
							 | 
						|
										ConsumerGroup: "test-group",
							 | 
						|
										ConsumerId:    "test-consumer",
							 | 
						|
										PartitionOffset: &schema_pb.PartitionOffset{
							 | 
						|
											Partition: partition,
							 | 
						|
										},
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Simulate the partition creation logic from SubscribeWithOffset
							 | 
						|
									p := topic.Partition{
							 | 
						|
										RingSize:   initMessage.PartitionOffset.Partition.RingSize,
							 | 
						|
										RangeStart: initMessage.PartitionOffset.Partition.RangeStart,
							 | 
						|
										RangeStop:  initMessage.PartitionOffset.Partition.RangeStop,
							 | 
						|
										UnixTimeNs: initMessage.PartitionOffset.Partition.UnixTimeNs,
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Verify that the partition preserves the original UnixTimeNs
							 | 
						|
									if p.UnixTimeNs != partition.UnixTimeNs {
							 | 
						|
										t.Errorf("Partition UnixTimeNs not preserved: expected %d, got %d",
							 | 
						|
											partition.UnixTimeNs, p.UnixTimeNs)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Verify partition key consistency
							 | 
						|
									expectedKey := fmt.Sprintf("ring:%d:range:%d-%d:time:%d",
							 | 
						|
										partition.RingSize, partition.RangeStart, partition.RangeStop, partition.UnixTimeNs)
							 | 
						|
								
							 | 
						|
									actualKey := fmt.Sprintf("ring:%d:range:%d-%d:time:%d",
							 | 
						|
										p.RingSize, p.RangeStart, p.RangeStop, p.UnixTimeNs)
							 | 
						|
								
							 | 
						|
									if actualKey != expectedKey {
							 | 
						|
										t.Errorf("Partition key mismatch: expected %s, got %s", expectedKey, actualKey)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								func TestBrokerOffsetManager_GetSubscription_Fixed(t *testing.T) {
							 | 
						|
									// Test that GetSubscription now works correctly after the fix
							 | 
						|
								
							 | 
						|
									storage := NewInMemoryOffsetStorageForTesting()
							 | 
						|
									offsetManager := NewBrokerOffsetManagerWithStorage(storage)
							 | 
						|
								
							 | 
						|
									// Create test topic and partition
							 | 
						|
									testTopic := topic.Topic{Namespace: "test", Name: "topic1"}
							 | 
						|
									testPartition := topic.Partition{
							 | 
						|
										RingSize:   32,
							 | 
						|
										RangeStart: 0,
							 | 
						|
										RangeStop:  31,
							 | 
						|
										UnixTimeNs: time.Now().UnixNano(),
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Test getting non-existent subscription
							 | 
						|
									_, err := offsetManager.GetSubscription("non-existent")
							 | 
						|
									if err == nil {
							 | 
						|
										t.Error("Expected error for non-existent subscription")
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Create a subscription
							 | 
						|
									subscriptionID := "test-subscription-fixed"
							 | 
						|
									subscription, err := offsetManager.CreateSubscription(
							 | 
						|
										subscriptionID,
							 | 
						|
										testTopic,
							 | 
						|
										testPartition,
							 | 
						|
										schema_pb.OffsetType_RESET_TO_EARLIEST,
							 | 
						|
										0,
							 | 
						|
									)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to create subscription: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Test getting existing subscription (this should now work)
							 | 
						|
									retrievedSub, err := offsetManager.GetSubscription(subscriptionID)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("GetSubscription failed after fix: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									if retrievedSub.ID != subscription.ID {
							 | 
						|
										t.Errorf("Expected subscription ID %s, got %s", subscription.ID, retrievedSub.ID)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									if retrievedSub.OffsetType != subscription.OffsetType {
							 | 
						|
										t.Errorf("Expected offset type %v, got %v", subscription.OffsetType, retrievedSub.OffsetType)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								func TestBrokerOffsetManager_ListActiveSubscriptions_Fixed(t *testing.T) {
							 | 
						|
									// Test that ListActiveSubscriptions now works correctly after the fix
							 | 
						|
								
							 | 
						|
									storage := NewInMemoryOffsetStorageForTesting()
							 | 
						|
									offsetManager := NewBrokerOffsetManagerWithStorage(storage)
							 | 
						|
								
							 | 
						|
									// Create test topic and partition
							 | 
						|
									testTopic := topic.Topic{Namespace: "test", Name: "topic1"}
							 | 
						|
									testPartition := topic.Partition{
							 | 
						|
										RingSize:   32,
							 | 
						|
										RangeStart: 0,
							 | 
						|
										RangeStop:  31,
							 | 
						|
										UnixTimeNs: time.Now().UnixNano(),
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Initially should have no subscriptions
							 | 
						|
									subscriptions, err := offsetManager.ListActiveSubscriptions()
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("ListActiveSubscriptions failed after fix: %v", err)
							 | 
						|
									}
							 | 
						|
									if len(subscriptions) != 0 {
							 | 
						|
										t.Errorf("Expected 0 subscriptions, got %d", len(subscriptions))
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Create multiple subscriptions (use RESET types to avoid HWM validation issues)
							 | 
						|
									subscriptionIDs := []string{"sub-fixed-1", "sub-fixed-2", "sub-fixed-3"}
							 | 
						|
									offsetTypes := []schema_pb.OffsetType{
							 | 
						|
										schema_pb.OffsetType_RESET_TO_EARLIEST,
							 | 
						|
										schema_pb.OffsetType_RESET_TO_LATEST,
							 | 
						|
										schema_pb.OffsetType_RESET_TO_EARLIEST, // Changed from EXACT_OFFSET
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									for i, subID := range subscriptionIDs {
							 | 
						|
										_, err := offsetManager.CreateSubscription(
							 | 
						|
											subID,
							 | 
						|
											testTopic,
							 | 
						|
											testPartition,
							 | 
						|
											offsetTypes[i],
							 | 
						|
											0, // Use 0 for all to avoid validation issues
							 | 
						|
										)
							 | 
						|
										if err != nil {
							 | 
						|
											t.Fatalf("Failed to create subscription %s: %v", subID, err)
							 | 
						|
										}
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// List all subscriptions (this should now work)
							 | 
						|
									subscriptions, err = offsetManager.ListActiveSubscriptions()
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("ListActiveSubscriptions failed after fix: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									if len(subscriptions) != len(subscriptionIDs) {
							 | 
						|
										t.Errorf("Expected %d subscriptions, got %d", len(subscriptionIDs), len(subscriptions))
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Verify all subscriptions are active
							 | 
						|
									for _, sub := range subscriptions {
							 | 
						|
										if !sub.IsActive {
							 | 
						|
											t.Errorf("Subscription %s should be active", sub.ID)
							 | 
						|
										}
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								func TestMessageQueueBroker_ListActiveSubscriptions_Fixed(t *testing.T) {
							 | 
						|
									// Test that the broker-level ListActiveSubscriptions now works correctly
							 | 
						|
								
							 | 
						|
									storage := NewInMemoryOffsetStorageForTesting()
							 | 
						|
									offsetManager := NewBrokerOffsetManagerWithStorage(storage)
							 | 
						|
								
							 | 
						|
									broker := &MessageQueueBroker{
							 | 
						|
										offsetManager: offsetManager,
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Create test topic and partition
							 | 
						|
									testTopic := topic.Topic{Namespace: "test", Name: "topic1"}
							 | 
						|
									testPartition := topic.Partition{
							 | 
						|
										RingSize:   32,
							 | 
						|
										RangeStart: 0,
							 | 
						|
										RangeStop:  31,
							 | 
						|
										UnixTimeNs: time.Now().UnixNano(),
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Initially should have no subscriptions
							 | 
						|
									subscriptionInfos, err := broker.ListActiveSubscriptions()
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Broker ListActiveSubscriptions failed after fix: %v", err)
							 | 
						|
									}
							 | 
						|
									if len(subscriptionInfos) != 0 {
							 | 
						|
										t.Errorf("Expected 0 subscription infos, got %d", len(subscriptionInfos))
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Create subscriptions with different offset types (use RESET types to avoid HWM validation issues)
							 | 
						|
									testCases := []struct {
							 | 
						|
										id          string
							 | 
						|
										offsetType  schema_pb.OffsetType
							 | 
						|
										startOffset int64
							 | 
						|
									}{
							 | 
						|
										{"broker-earliest-sub", schema_pb.OffsetType_RESET_TO_EARLIEST, 0},
							 | 
						|
										{"broker-latest-sub", schema_pb.OffsetType_RESET_TO_LATEST, 0},
							 | 
						|
										{"broker-reset-sub", schema_pb.OffsetType_RESET_TO_EARLIEST, 0}, // Changed from EXACT_OFFSET
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									for _, tc := range testCases {
							 | 
						|
										_, err := broker.offsetManager.CreateSubscription(
							 | 
						|
											tc.id,
							 | 
						|
											testTopic,
							 | 
						|
											testPartition,
							 | 
						|
											tc.offsetType,
							 | 
						|
											tc.startOffset,
							 | 
						|
										)
							 | 
						|
										if err != nil {
							 | 
						|
											t.Fatalf("Failed to create subscription %s: %v", tc.id, err)
							 | 
						|
										}
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// List subscription infos (this should now work)
							 | 
						|
									subscriptionInfos, err = broker.ListActiveSubscriptions()
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Broker ListActiveSubscriptions failed after fix: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									if len(subscriptionInfos) != len(testCases) {
							 | 
						|
										t.Errorf("Expected %d subscription infos, got %d", len(testCases), len(subscriptionInfos))
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Verify subscription info structure
							 | 
						|
									for _, info := range subscriptionInfos {
							 | 
						|
										// Check required fields
							 | 
						|
										requiredFields := []string{
							 | 
						|
											"subscription_id", "start_offset", "current_offset",
							 | 
						|
											"offset_type", "is_active", "lag", "at_end",
							 | 
						|
										}
							 | 
						|
								
							 | 
						|
										for _, field := range requiredFields {
							 | 
						|
											if _, ok := info[field]; !ok {
							 | 
						|
												t.Errorf("Missing field %s in subscription info", field)
							 | 
						|
											}
							 | 
						|
										}
							 | 
						|
								
							 | 
						|
										// Verify is_active is true
							 | 
						|
										if isActive, ok := info["is_active"].(bool); !ok || !isActive {
							 | 
						|
											t.Errorf("Expected is_active to be true, got %v", info["is_active"])
							 | 
						|
										}
							 | 
						|
								
							 | 
						|
									}
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								func TestSingleWriterPerPartitionCorrectness(t *testing.T) {
							 | 
						|
									// Test that demonstrates correctness under single-writer-per-partition model
							 | 
						|
								
							 | 
						|
									// Simulate two brokers with separate offset managers but same partition
							 | 
						|
									storage1 := NewInMemoryOffsetStorageForTesting()
							 | 
						|
									storage2 := NewInMemoryOffsetStorageForTesting()
							 | 
						|
								
							 | 
						|
									offsetManager1 := NewBrokerOffsetManagerWithStorage(storage1)
							 | 
						|
									offsetManager2 := NewBrokerOffsetManagerWithStorage(storage2)
							 | 
						|
								
							 | 
						|
									broker1 := &MessageQueueBroker{offsetManager: offsetManager1}
							 | 
						|
									broker2 := &MessageQueueBroker{offsetManager: offsetManager2}
							 | 
						|
								
							 | 
						|
									// Same partition identity (this is key for correctness)
							 | 
						|
									fixedTimestamp := time.Now().UnixNano()
							 | 
						|
									testTopic := topic.Topic{Namespace: "test", Name: "shared-topic"}
							 | 
						|
									testPartition := topic.Partition{
							 | 
						|
										RingSize:   32,
							 | 
						|
										RangeStart: 0,
							 | 
						|
										RangeStop:  31,
							 | 
						|
										UnixTimeNs: fixedTimestamp, // Same timestamp = same partition identity
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Broker 1 is the leader for this partition - assigns offsets
							 | 
						|
									baseOffset, lastOffset, err := broker1.offsetManager.AssignBatchOffsets(testTopic, testPartition, 10)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to assign offsets on broker1: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									if baseOffset != 0 || lastOffset != 9 {
							 | 
						|
										t.Errorf("Expected offsets 0-9, got %d-%d", baseOffset, lastOffset)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Get HWM from leader
							 | 
						|
									hwm1, err := broker1.offsetManager.GetHighWaterMark(testTopic, testPartition)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to get HWM from broker1: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									if hwm1 != 10 {
							 | 
						|
										t.Errorf("Expected HWM 10 on leader, got %d", hwm1)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Broker 2 is a follower - should have HWM 0 (no local assignments)
							 | 
						|
									hwm2, err := broker2.offsetManager.GetHighWaterMark(testTopic, testPartition)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to get HWM from broker2: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									if hwm2 != 0 {
							 | 
						|
										t.Errorf("Expected HWM 0 on follower, got %d", hwm2)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Create subscription on leader (where offsets were assigned)
							 | 
						|
									subscription1, err := broker1.offsetManager.CreateSubscription(
							 | 
						|
										"leader-subscription",
							 | 
						|
										testTopic,
							 | 
						|
										testPartition,
							 | 
						|
										schema_pb.OffsetType_RESET_TO_EARLIEST,
							 | 
						|
										0,
							 | 
						|
									)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to create subscription on leader: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Verify subscription can see the correct HWM
							 | 
						|
									lag1, err := subscription1.GetLag()
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to get lag on leader subscription: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									if lag1 != 10 {
							 | 
						|
										t.Errorf("Expected lag 10 on leader subscription, got %d", lag1)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Create subscription on follower (should have different lag due to local HWM)
							 | 
						|
									subscription2, err := broker2.offsetManager.CreateSubscription(
							 | 
						|
										"follower-subscription",
							 | 
						|
										testTopic,
							 | 
						|
										testPartition,
							 | 
						|
										schema_pb.OffsetType_RESET_TO_EARLIEST,
							 | 
						|
										0,
							 | 
						|
									)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to create subscription on follower: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									lag2, err := subscription2.GetLag()
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to get lag on follower subscription: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									if lag2 != 0 {
							 | 
						|
										t.Errorf("Expected lag 0 on follower subscription (no local data), got %d", lag2)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								func TestEndToEndWorkflowAfterFixes(t *testing.T) {
							 | 
						|
									// Test the complete workflow with all fixes applied
							 | 
						|
								
							 | 
						|
									storage := NewInMemoryOffsetStorageForTesting()
							 | 
						|
									offsetManager := NewBrokerOffsetManagerWithStorage(storage)
							 | 
						|
								
							 | 
						|
									broker := &MessageQueueBroker{
							 | 
						|
										offsetManager: offsetManager,
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// Create test topic and partition with fixed timestamp
							 | 
						|
									fixedTimestamp := time.Now().UnixNano()
							 | 
						|
									testTopic := topic.Topic{Namespace: "test", Name: "e2e-topic"}
							 | 
						|
									testPartition := topic.Partition{
							 | 
						|
										RingSize:   32,
							 | 
						|
										RangeStart: 0,
							 | 
						|
										RangeStop:  31,
							 | 
						|
										UnixTimeNs: fixedTimestamp,
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									subscriptionID := "e2e-test-sub"
							 | 
						|
								
							 | 
						|
									// 1. Create subscription (use RESET_TO_EARLIEST to avoid HWM validation issues)
							 | 
						|
									subscription, err := broker.offsetManager.CreateSubscription(
							 | 
						|
										subscriptionID,
							 | 
						|
										testTopic,
							 | 
						|
										testPartition,
							 | 
						|
										schema_pb.OffsetType_RESET_TO_EARLIEST,
							 | 
						|
										0,
							 | 
						|
									)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to create subscription: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// 2. Verify GetSubscription works
							 | 
						|
									retrievedSub, err := broker.offsetManager.GetSubscription(subscriptionID)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("GetSubscription failed: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									if retrievedSub.ID != subscription.ID {
							 | 
						|
										t.Errorf("GetSubscription returned wrong subscription: expected %s, got %s",
							 | 
						|
											subscription.ID, retrievedSub.ID)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// 3. Verify it appears in active list
							 | 
						|
									activeList, err := broker.ListActiveSubscriptions()
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to list active subscriptions: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									found := false
							 | 
						|
									for _, info := range activeList {
							 | 
						|
										if info["subscription_id"] == subscriptionID {
							 | 
						|
											found = true
							 | 
						|
											break
							 | 
						|
										}
							 | 
						|
									}
							 | 
						|
									if !found {
							 | 
						|
										t.Error("New subscription not found in active list")
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// 4. Get subscription info
							 | 
						|
									info, err := broker.GetSubscriptionInfo(subscriptionID)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to get subscription info: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									if info["subscription_id"] != subscriptionID {
							 | 
						|
										t.Errorf("Wrong subscription ID in info: expected %s, got %v", subscriptionID, info["subscription_id"])
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// 5. Assign some offsets to create data for seeking
							 | 
						|
									_, _, err = broker.offsetManager.AssignBatchOffsets(testTopic, testPartition, 50)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to assign offsets: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// 6. Seek subscription
							 | 
						|
									newOffset := int64(42)
							 | 
						|
									err = broker.SeekSubscription(subscriptionID, newOffset)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to seek subscription: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// 7. Verify seek worked
							 | 
						|
									updatedInfo, err := broker.GetSubscriptionInfo(subscriptionID)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to get updated subscription info: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									if updatedInfo["current_offset"] != newOffset {
							 | 
						|
										t.Errorf("Seek didn't work: expected offset %d, got %v", newOffset, updatedInfo["current_offset"])
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// 8. Test offset to timestamp conversion with fixed partition identity
							 | 
						|
									updatedSub, err := broker.offsetManager.GetSubscription(subscriptionID)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to get updated subscription: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									position, err := broker.convertOffsetToMessagePosition(updatedSub)
							 | 
						|
									if err != nil {
							 | 
						|
										t.Fatalf("Failed to convert offset to position: %v", err)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									if position.Time.IsZero() {
							 | 
						|
										t.Error("Expected non-zero timestamp from conversion")
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
									// 9. Verify partition identity consistency throughout
							 | 
						|
									partitionKey1 := fmt.Sprintf("ring:%d:range:%d-%d:time:%d",
							 | 
						|
										testPartition.RingSize, testPartition.RangeStart, testPartition.RangeStop, testPartition.UnixTimeNs)
							 | 
						|
								
							 | 
						|
									partitionKey2 := fmt.Sprintf("ring:%d:range:%d-%d:time:%d",
							 | 
						|
										testPartition.RingSize, testPartition.RangeStart, testPartition.RangeStop, fixedTimestamp)
							 | 
						|
								
							 | 
						|
									if partitionKey1 != partitionKey2 {
							 | 
						|
										t.Errorf("Partition key inconsistency: %s != %s", partitionKey1, partitionKey2)
							 | 
						|
									}
							 | 
						|
								
							 | 
						|
								}
							 |