|
|
@ -24,22 +24,24 @@ import ( |
|
|
|
"google.golang.org/protobuf/proto" |
|
|
|
) |
|
|
|
|
|
|
|
// HybridMessageScanner scans both live message log files AND archived Parquet files
|
|
|
|
// HybridMessageScanner scans from ALL data sources:
|
|
|
|
// Architecture:
|
|
|
|
// 1. Recent/live messages stored in log files (filer_pb.LogEntry format)
|
|
|
|
// 2. Older messages archived to Parquet files (schema_pb.RecordValue format)
|
|
|
|
// 3. Seamlessly merges data from both sources chronologically
|
|
|
|
// 4. Provides complete view of all messages in a topic
|
|
|
|
// 1. Unflushed in-memory data from brokers (mq_pb.DataMessage format) - REAL-TIME
|
|
|
|
// 2. Recent/live messages in log files (filer_pb.LogEntry format) - FLUSHED
|
|
|
|
// 3. Older messages in Parquet files (schema_pb.RecordValue format) - ARCHIVED
|
|
|
|
// 4. Seamlessly merges data from all sources chronologically
|
|
|
|
// 5. Provides complete real-time view of all messages in a topic
|
|
|
|
type HybridMessageScanner struct { |
|
|
|
filerClient filer_pb.FilerClient |
|
|
|
brokerClient BrokerClientInterface // For querying unflushed data
|
|
|
|
topic topic.Topic |
|
|
|
recordSchema *schema_pb.RecordType |
|
|
|
parquetLevels *schema.ParquetLevels |
|
|
|
} |
|
|
|
|
|
|
|
// NewHybridMessageScanner creates a scanner that reads from both live logs and Parquet files
|
|
|
|
// This replaces ParquetScanner to provide complete message coverage
|
|
|
|
func NewHybridMessageScanner(filerClient filer_pb.FilerClient, namespace, topicName string) (*HybridMessageScanner, error) { |
|
|
|
// NewHybridMessageScanner creates a scanner that reads from all data sources
|
|
|
|
// This provides complete real-time message coverage including unflushed data
|
|
|
|
func NewHybridMessageScanner(filerClient filer_pb.FilerClient, brokerClient BrokerClientInterface, namespace, topicName string) (*HybridMessageScanner, error) { |
|
|
|
// Check if filerClient is available
|
|
|
|
if filerClient == nil { |
|
|
|
return nil, fmt.Errorf("filerClient is required but not available") |
|
|
@ -81,6 +83,7 @@ func NewHybridMessageScanner(filerClient filer_pb.FilerClient, namespace, topicN |
|
|
|
|
|
|
|
return &HybridMessageScanner{ |
|
|
|
filerClient: filerClient, |
|
|
|
brokerClient: brokerClient, |
|
|
|
topic: t, |
|
|
|
recordSchema: recordType, |
|
|
|
parquetLevels: parquetLevels, |
|
|
@ -162,6 +165,88 @@ func (hms *HybridMessageScanner) Scan(ctx context.Context, options HybridScanOpt |
|
|
|
return results, nil |
|
|
|
} |
|
|
|
|
|
|
|
// scanUnflushedData queries brokers for unflushed in-memory data
|
|
|
|
func (hms *HybridMessageScanner) scanUnflushedData(ctx context.Context, partition topic.Partition, options HybridScanOptions) ([]HybridScanResult, error) { |
|
|
|
var results []HybridScanResult |
|
|
|
|
|
|
|
// Skip if no broker client available
|
|
|
|
if hms.brokerClient == nil { |
|
|
|
return results, nil |
|
|
|
} |
|
|
|
|
|
|
|
// Get broker address for this partition
|
|
|
|
// TODO: Implement proper broker discovery for partition
|
|
|
|
// For now, assume broker client knows how to reach the right broker
|
|
|
|
|
|
|
|
// Create a temporary slice to collect unflushed messages
|
|
|
|
unflushedMessages := make([]*mq_pb.DataMessage, 0) |
|
|
|
|
|
|
|
// We need to call the broker to get unflushed data
|
|
|
|
// For now, we'll implement this as a best-effort approach
|
|
|
|
// In a full implementation, this would require a new gRPC method on the broker
|
|
|
|
// TODO: Implement actual broker gRPC call to get unflushed data
|
|
|
|
|
|
|
|
// Convert unflushed messages to HybridScanResult format
|
|
|
|
for _, msg := range unflushedMessages { |
|
|
|
// Skip messages outside time range
|
|
|
|
if options.StartTimeNs > 0 && msg.TsNs < options.StartTimeNs { |
|
|
|
continue |
|
|
|
} |
|
|
|
if options.StopTimeNs > 0 && msg.TsNs > options.StopTimeNs { |
|
|
|
continue |
|
|
|
} |
|
|
|
|
|
|
|
// Convert DataMessage to RecordValue format
|
|
|
|
recordValue, _, err := hms.convertDataMessageToRecord(msg) |
|
|
|
if err != nil { |
|
|
|
continue // Skip malformed messages
|
|
|
|
} |
|
|
|
|
|
|
|
// Apply predicate filter if provided
|
|
|
|
if options.Predicate != nil && !options.Predicate(recordValue) { |
|
|
|
continue |
|
|
|
} |
|
|
|
|
|
|
|
// Convert to HybridScanResult
|
|
|
|
result := HybridScanResult{ |
|
|
|
Values: recordValue.Fields, |
|
|
|
Timestamp: msg.TsNs, |
|
|
|
Key: msg.Key, |
|
|
|
Source: "in_memory_broker", |
|
|
|
} |
|
|
|
|
|
|
|
results = append(results, result) |
|
|
|
|
|
|
|
// Apply limit
|
|
|
|
if options.Limit > 0 && len(results) >= options.Limit { |
|
|
|
break |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
return results, nil |
|
|
|
} |
|
|
|
|
|
|
|
// convertDataMessageToRecord converts mq_pb.DataMessage to schema_pb.RecordValue
|
|
|
|
func (hms *HybridMessageScanner) convertDataMessageToRecord(msg *mq_pb.DataMessage) (*schema_pb.RecordValue, string, error) { |
|
|
|
// Parse the message data as RecordValue
|
|
|
|
recordValue := &schema_pb.RecordValue{} |
|
|
|
if err := proto.Unmarshal(msg.Value, recordValue); err != nil { |
|
|
|
return nil, "", fmt.Errorf("failed to unmarshal message data: %v", err) |
|
|
|
} |
|
|
|
|
|
|
|
// Add system columns
|
|
|
|
if recordValue.Fields == nil { |
|
|
|
recordValue.Fields = make(map[string]*schema_pb.Value) |
|
|
|
} |
|
|
|
|
|
|
|
// Add timestamp
|
|
|
|
recordValue.Fields[SW_COLUMN_NAME_TS] = &schema_pb.Value{ |
|
|
|
Kind: &schema_pb.Value_Int64Value{Int64Value: msg.TsNs}, |
|
|
|
} |
|
|
|
|
|
|
|
return recordValue, string(msg.Key), nil |
|
|
|
} |
|
|
|
|
|
|
|
// discoverTopicPartitions discovers the actual partitions for this topic by scanning the filesystem
|
|
|
|
// This finds real partition directories like v2025-09-01-07-16-34/0000-0630/
|
|
|
|
func (hms *HybridMessageScanner) discoverTopicPartitions(ctx context.Context) ([]topic.Partition, error) { |
|
|
@ -227,10 +312,22 @@ func (hms *HybridMessageScanner) discoverTopicPartitions(ctx context.Context) ([ |
|
|
|
} |
|
|
|
|
|
|
|
// scanPartitionHybrid scans a specific partition using the hybrid approach
|
|
|
|
// This is where the magic happens - seamlessly reading live + archived data
|
|
|
|
// This is where the magic happens - seamlessly reading ALL data sources:
|
|
|
|
// 1. Unflushed in-memory data from brokers (REAL-TIME)
|
|
|
|
// 2. Live logs + Parquet files from disk (FLUSHED/ARCHIVED)
|
|
|
|
func (hms *HybridMessageScanner) scanPartitionHybrid(ctx context.Context, partition topic.Partition, options HybridScanOptions) ([]HybridScanResult, error) { |
|
|
|
var results []HybridScanResult |
|
|
|
|
|
|
|
// STEP 1: Scan unflushed in-memory data from brokers (REAL-TIME)
|
|
|
|
unflushedResults, err := hms.scanUnflushedData(ctx, partition, options) |
|
|
|
if err != nil { |
|
|
|
// Don't fail the query if broker scanning fails - just log and continue with disk data
|
|
|
|
fmt.Printf("Warning: Failed to scan unflushed data from broker: %v\n", err) |
|
|
|
} else { |
|
|
|
results = append(results, unflushedResults...) |
|
|
|
} |
|
|
|
|
|
|
|
// STEP 2: Scan flushed data from disk (live logs + Parquet files)
|
|
|
|
// Create the hybrid read function that combines live logs + Parquet files
|
|
|
|
// This uses SeaweedFS MQ's own merged reading logic
|
|
|
|
mergedReadFn := logstore.GenMergedReadFunc(hms.filerClient, hms.topic, partition) |
|
|
@ -296,12 +393,41 @@ func (hms *HybridMessageScanner) scanPartitionHybrid(ctx context.Context, partit |
|
|
|
return false, nil |
|
|
|
} |
|
|
|
|
|
|
|
// Start scanning from the specified position
|
|
|
|
startPosition := log_buffer.MessagePosition{Time: startTime} |
|
|
|
_, _, err := mergedReadFn(startPosition, stopTsNs, eachLogEntryFn) |
|
|
|
// Only scan flushed data if we haven't reached the limit from unflushed data
|
|
|
|
if options.Limit == 0 || len(results) < options.Limit { |
|
|
|
// Adjust limit for remaining capacity
|
|
|
|
remainingLimit := options.Limit - len(results) |
|
|
|
if remainingLimit > 0 { |
|
|
|
// Create a copy of options with adjusted limit for flushed data
|
|
|
|
flushedOptions := options |
|
|
|
flushedOptions.Limit = remainingLimit |
|
|
|
} |
|
|
|
|
|
|
|
// Start scanning from the specified position
|
|
|
|
startPosition := log_buffer.MessagePosition{Time: startTime} |
|
|
|
_, _, err = mergedReadFn(startPosition, stopTsNs, eachLogEntryFn) |
|
|
|
|
|
|
|
if err != nil { |
|
|
|
return nil, fmt.Errorf("hybrid scan failed: %v", err) |
|
|
|
if err != nil { |
|
|
|
return nil, fmt.Errorf("flushed data scan failed: %v", err) |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
// STEP 3: Sort results chronologically (unflushed + flushed data)
|
|
|
|
// This ensures proper time ordering across all data sources
|
|
|
|
if len(results) > 1 { |
|
|
|
// Simple sort by timestamp - in a full implementation, consider more efficient merging
|
|
|
|
for i := 0; i < len(results)-1; i++ { |
|
|
|
for j := i + 1; j < len(results); j++ { |
|
|
|
if results[i].Timestamp > results[j].Timestamp { |
|
|
|
results[i], results[j] = results[j], results[i] |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
} |
|
|
|
|
|
|
|
// Apply final limit after merging and sorting
|
|
|
|
if options.Limit > 0 && len(results) > options.Limit { |
|
|
|
results = results[:options.Limit] |
|
|
|
} |
|
|
|
|
|
|
|
return results, nil |
|
|
|