Browse Source

fix TestSaramaProduceConsume

pull/7231/head
chrislu 2 months ago
parent
commit
964d1d06e4
  1. 141
      weed/mq/kafka/protocol/fetch.go
  2. 61
      weed/mq/kafka/protocol/handler.go
  3. 220
      weed/mq/kafka/protocol/produce.go

141
weed/mq/kafka/protocol/fetch.go

@ -108,10 +108,27 @@ func (h *Handler) handleFetch(correlationID uint32, apiVersion uint16, requestBo
fmt.Printf("DEBUG: GetRecordBatch delegated to SeaweedMQ handler - topic:%s, partition:%d, offset:%d\n",
topic.Name, partition.PartitionID, partition.FetchOffset)
// Use synthetic record batch since we don't have stored batches yet
fmt.Printf("DEBUG: No stored record batch found for offset %d, using synthetic batch\n", partition.FetchOffset)
// Try to get stored messages first
if basicHandler, ok := h.seaweedMQHandler.(*basicSeaweedMQHandler); ok {
maxMessages := int(highWaterMark - partition.FetchOffset)
if maxMessages > 10 {
maxMessages = 10
}
storedMessages := basicHandler.GetStoredMessages(topic.Name, partition.PartitionID, partition.FetchOffset, maxMessages)
if len(storedMessages) > 0 {
fmt.Printf("DEBUG: Found %d stored messages for offset %d, constructing real record batch\n", len(storedMessages), partition.FetchOffset)
recordBatch = h.constructRecordBatchFromMessages(partition.FetchOffset, storedMessages)
fmt.Printf("DEBUG: Using real stored message batch for offset %d, size: %d bytes\n", partition.FetchOffset, len(recordBatch))
} else {
fmt.Printf("DEBUG: No stored messages found for offset %d, using synthetic batch\n", partition.FetchOffset)
recordBatch = h.constructSimpleRecordBatch(partition.FetchOffset, highWaterMark)
fmt.Printf("DEBUG: Using synthetic record batch for offset %d, size: %d bytes\n", partition.FetchOffset, len(recordBatch))
}
} else {
fmt.Printf("DEBUG: Not using basicSeaweedMQHandler, using synthetic batch\n")
recordBatch = h.constructSimpleRecordBatch(partition.FetchOffset, highWaterMark)
fmt.Printf("DEBUG: Using synthetic record batch for offset %d, size: %d bytes\n", partition.FetchOffset, len(recordBatch))
}
} else {
fmt.Printf("DEBUG: No messages available - fetchOffset %d >= highWaterMark %d\n", partition.FetchOffset, highWaterMark)
recordBatch = []byte{} // No messages available
@ -571,6 +588,126 @@ func (h *Handler) constructSimpleRecordBatch(fetchOffset, highWaterMark int64) [
return batch
}
// constructRecordBatchFromMessages creates a Kafka record batch from actual stored messages
func (h *Handler) constructRecordBatchFromMessages(fetchOffset int64, messages []*MessageRecord) []byte {
if len(messages) == 0 {
return []byte{}
}
// Create record batch using the real stored messages
batch := make([]byte, 0, 512)
// Record batch header
baseOffsetBytes := make([]byte, 8)
binary.BigEndian.PutUint64(baseOffsetBytes, uint64(fetchOffset))
batch = append(batch, baseOffsetBytes...) // base offset (8 bytes)
// Calculate batch length (will be filled after we know the size)
batchLengthPos := len(batch)
batch = append(batch, 0, 0, 0, 0) // batch length placeholder (4 bytes)
// Partition leader epoch (4 bytes) - use -1 for no epoch
batch = append(batch, 0xFF, 0xFF, 0xFF, 0xFF)
// Magic byte (1 byte) - v2 format
batch = append(batch, 2)
// CRC placeholder (4 bytes) - will be calculated later
crcPos := len(batch)
batch = append(batch, 0, 0, 0, 0)
// Attributes (2 bytes) - no compression, etc.
batch = append(batch, 0, 0)
// Last offset delta (4 bytes)
lastOffsetDelta := int32(len(messages) - 1)
lastOffsetDeltaBytes := make([]byte, 4)
binary.BigEndian.PutUint32(lastOffsetDeltaBytes, uint32(lastOffsetDelta))
batch = append(batch, lastOffsetDeltaBytes...)
// Base timestamp (8 bytes) - use first message timestamp
baseTimestamp := messages[0].Timestamp
baseTimestampBytes := make([]byte, 8)
binary.BigEndian.PutUint64(baseTimestampBytes, uint64(baseTimestamp))
batch = append(batch, baseTimestampBytes...)
// Max timestamp (8 bytes) - use last message timestamp or same as base
maxTimestamp := baseTimestamp
if len(messages) > 1 {
maxTimestamp = messages[len(messages)-1].Timestamp
}
maxTimestampBytes := make([]byte, 8)
binary.BigEndian.PutUint64(maxTimestampBytes, uint64(maxTimestamp))
batch = append(batch, maxTimestampBytes...)
// Producer ID (8 bytes) - use -1 for no producer ID
batch = append(batch, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF)
// Producer epoch (2 bytes) - use -1 for no producer epoch
batch = append(batch, 0xFF, 0xFF)
// Base sequence (4 bytes) - use -1 for no base sequence
batch = append(batch, 0xFF, 0xFF, 0xFF, 0xFF)
// Records count (4 bytes)
recordCountBytes := make([]byte, 4)
binary.BigEndian.PutUint32(recordCountBytes, uint32(len(messages)))
batch = append(batch, recordCountBytes...)
// Add individual records from stored messages
for i, msg := range messages {
// Build individual record
record := make([]byte, 0, 128)
// Record attributes (1 byte)
record = append(record, 0)
// Timestamp delta (varint) - calculate from base timestamp
timestampDelta := msg.Timestamp - baseTimestamp
record = append(record, encodeVarint(timestampDelta)...)
// Offset delta (varint)
offsetDelta := int64(i)
record = append(record, encodeVarint(offsetDelta)...)
// Key length and key (varint + data)
if msg.Key == nil {
record = append(record, encodeVarint(-1)...) // null key
} else {
record = append(record, encodeVarint(int64(len(msg.Key)))...)
record = append(record, msg.Key...)
}
// Value length and value (varint + data)
if msg.Value == nil {
record = append(record, encodeVarint(-1)...) // null value
} else {
record = append(record, encodeVarint(int64(len(msg.Value)))...)
record = append(record, msg.Value...)
}
// Headers count (varint) - 0 headers
record = append(record, encodeVarint(0)...)
// Prepend record length (varint)
recordLength := int64(len(record))
batch = append(batch, encodeVarint(recordLength)...)
batch = append(batch, record...)
}
// Fill in the batch length
batchLength := uint32(len(batch) - batchLengthPos - 4)
binary.BigEndian.PutUint32(batch[batchLengthPos:batchLengthPos+4], batchLength)
// Calculate CRC32 for the batch
crcStartPos := crcPos + 4 // start after the CRC field
crcData := batch[crcStartPos:]
crc := crc32.Checksum(crcData, crc32.MakeTable(crc32.Castagnoli))
binary.BigEndian.PutUint32(batch[crcPos:crcPos+4], crc)
return batch
}
// constructRecordBatch creates a realistic Kafka record batch that matches produced messages
// This creates record batches that mirror what was actually stored during Produce operations
func (h *Handler) constructRecordBatch(ledger interface{}, fetchOffset, highWaterMark int64) []byte {

61
weed/mq/kafka/protocol/handler.go

@ -74,6 +74,7 @@ func NewHandler() *Handler {
seaweedMQHandler: &basicSeaweedMQHandler{
topics: make(map[string]bool),
ledgers: make(map[string]*offset.Ledger),
messages: make(map[string]map[int32]map[int64]*MessageRecord),
},
}
}
@ -92,10 +93,19 @@ func NewTestHandler() *Handler {
}
}
// MessageRecord represents a stored message
type MessageRecord struct {
Key []byte
Value []byte
Timestamp int64
}
// basicSeaweedMQHandler is a minimal in-memory implementation for basic Kafka functionality
type basicSeaweedMQHandler struct {
topics map[string]bool
ledgers map[string]*offset.Ledger
// messages stores actual message content indexed by topic-partition-offset
messages map[string]map[int32]map[int64]*MessageRecord // topic -> partition -> offset -> message
mu sync.RWMutex
}
@ -162,9 +172,13 @@ func (b *basicSeaweedMQHandler) GetLedger(topic string, partition int32) *offset
}
func (b *basicSeaweedMQHandler) ProduceRecord(topicName string, partitionID int32, key, value []byte) (int64, error) {
// Store the record in the ledger
// Get or create the ledger first (this will acquire and release the lock)
ledger := b.GetOrCreateLedger(topicName, partitionID)
// Now acquire the lock for the rest of the operation
b.mu.Lock()
defer b.mu.Unlock()
// Assign an offset and append the record
offset := ledger.AssignOffsets(1)
timestamp := time.Now().UnixNano()
@ -174,9 +188,54 @@ func (b *basicSeaweedMQHandler) ProduceRecord(topicName string, partitionID int3
return 0, fmt.Errorf("failed to append record: %w", err)
}
// Store the actual message content
if b.messages[topicName] == nil {
b.messages[topicName] = make(map[int32]map[int64]*MessageRecord)
}
if b.messages[topicName][partitionID] == nil {
b.messages[topicName][partitionID] = make(map[int64]*MessageRecord)
}
// Make copies of key and value to avoid referencing the original slices
keyCopy := make([]byte, len(key))
copy(keyCopy, key)
valueCopy := make([]byte, len(value))
copy(valueCopy, value)
b.messages[topicName][partitionID][offset] = &MessageRecord{
Key: keyCopy,
Value: valueCopy,
Timestamp: timestamp,
}
return offset, nil
}
// GetStoredMessages retrieves stored messages for a topic-partition from a given offset
func (b *basicSeaweedMQHandler) GetStoredMessages(topicName string, partitionID int32, fromOffset int64, maxMessages int) []*MessageRecord {
b.mu.RLock()
defer b.mu.RUnlock()
if b.messages[topicName] == nil || b.messages[topicName][partitionID] == nil {
return nil
}
partitionMessages := b.messages[topicName][partitionID]
var result []*MessageRecord
// Collect messages starting from fromOffset
for offset := fromOffset; offset < fromOffset+int64(maxMessages); offset++ {
if msg, exists := partitionMessages[offset]; exists {
result = append(result, msg)
} else {
// No more consecutive messages
break
}
}
return result
}
func (b *basicSeaweedMQHandler) Close() error {
return nil
}

220
weed/mq/kafka/protocol/produce.go

@ -267,27 +267,221 @@ func (h *Handler) produceToSeaweedMQ(topic string, partition int32, recordSetDat
return h.seaweedMQHandler.ProduceRecord(topic, partition, key, value)
}
// extractFirstRecord extracts the first record from a Kafka record set (simplified)
// TODO: CRITICAL - This function returns placeholder data instead of parsing real records
// For real client compatibility, need to:
// - Parse record batch header properly
// - Extract actual key/value from first record in batch
// - Handle compressed record batches
// - Support all record formats (v0, v1, v2)
// extractFirstRecord extracts the first record from a Kafka record batch
func (h *Handler) extractFirstRecord(recordSetData []byte) ([]byte, []byte) {
// For Phase 2, create a simple placeholder record
// This represents what would be extracted from the actual Kafka record batch
fmt.Printf("DEBUG: extractFirstRecord - parsing %d bytes\n", len(recordSetData))
if len(recordSetData) < 61 {
fmt.Printf("DEBUG: Record batch too short (%d bytes), returning placeholder\n", len(recordSetData))
// Fallback to placeholder
key := []byte("kafka-key")
value := fmt.Sprintf("kafka-message-data-%d", time.Now().UnixNano())
return key, []byte(value)
}
offset := 0
// Parse record batch header (Kafka v2 format)
// base_offset(8) + batch_length(4) + partition_leader_epoch(4) + magic(1) + crc(4) + attributes(2)
// + last_offset_delta(4) + first_timestamp(8) + max_timestamp(8) + producer_id(8) + producer_epoch(2)
// + base_sequence(4) + records_count(4) = 61 bytes header
offset += 8 // skip base_offset
batchLength := int32(binary.BigEndian.Uint32(recordSetData[offset:]))
offset += 4 // batch_length
fmt.Printf("DEBUG: Record batch length: %d\n", batchLength)
offset += 4 // skip partition_leader_epoch
magic := recordSetData[offset]
offset += 1 // magic byte
fmt.Printf("DEBUG: Magic byte: %d\n", magic)
if magic != 2 {
fmt.Printf("DEBUG: Unsupported magic byte %d, returning placeholder\n", magic)
// Fallback for older formats
key := []byte("kafka-key")
value := fmt.Sprintf("kafka-message-data-%d", time.Now().UnixNano())
return key, []byte(value)
}
offset += 4 // skip crc
offset += 2 // skip attributes
offset += 4 // skip last_offset_delta
offset += 8 // skip first_timestamp
offset += 8 // skip max_timestamp
offset += 8 // skip producer_id
offset += 2 // skip producer_epoch
offset += 4 // skip base_sequence
recordsCount := int32(binary.BigEndian.Uint32(recordSetData[offset:]))
offset += 4 // records_count
fmt.Printf("DEBUG: Records count: %d\n", recordsCount)
if recordsCount == 0 {
fmt.Printf("DEBUG: No records in batch, returning placeholder\n")
key := []byte("kafka-key")
value := fmt.Sprintf("kafka-message-data-%d", time.Now().UnixNano())
return key, []byte(value)
}
// Parse first record
if offset >= len(recordSetData) {
fmt.Printf("DEBUG: Record data truncated at offset %d, returning placeholder\n", offset)
key := []byte("kafka-key")
value := fmt.Sprintf("kafka-message-data-%d", time.Now().UnixNano())
return key, []byte(value)
}
// Read record length (varint)
recordLength, varintLen := decodeVarint(recordSetData[offset:])
if varintLen == 0 {
fmt.Printf("DEBUG: Failed to decode record length varint, returning placeholder\n")
key := []byte("kafka-key")
value := fmt.Sprintf("kafka-message-data-%d", time.Now().UnixNano())
return key, []byte(value)
}
offset += varintLen
fmt.Printf("DEBUG: First record length: %d\n", recordLength)
if offset+int(recordLength) > len(recordSetData) {
fmt.Printf("DEBUG: Record data extends beyond batch, returning placeholder\n")
key := []byte("kafka-key")
value := fmt.Sprintf("kafka-message-data-%d", time.Now().UnixNano())
return key, []byte(value)
}
// In a real implementation, this would:
// 1. Parse the record batch header
// 2. Extract individual records with proper key/value/timestamp
// 3. Handle compression, transaction markers, etc.
recordData := recordSetData[offset : offset+int(recordLength)]
recordOffset := 0
// Parse record: attributes(1) + timestamp_delta(varint) + offset_delta(varint) + key + value + headers
recordOffset += 1 // skip attributes
// Skip timestamp_delta (varint)
_, varintLen = decodeVarint(recordData[recordOffset:])
if varintLen == 0 {
fmt.Printf("DEBUG: Failed to decode timestamp delta, returning placeholder\n")
key := []byte("kafka-key")
value := fmt.Sprintf("kafka-message-data-%d", time.Now().UnixNano())
return key, []byte(value)
}
recordOffset += varintLen
// Skip offset_delta (varint)
_, varintLen = decodeVarint(recordData[recordOffset:])
if varintLen == 0 {
fmt.Printf("DEBUG: Failed to decode offset delta, returning placeholder\n")
key := []byte("kafka-key")
value := fmt.Sprintf("kafka-message-data-%d", time.Now().UnixNano())
return key, []byte(value)
}
recordOffset += varintLen
// Read key length and key
keyLength, varintLen := decodeVarint(recordData[recordOffset:])
if varintLen == 0 {
fmt.Printf("DEBUG: Failed to decode key length, returning placeholder\n")
key := []byte("kafka-key")
value := fmt.Sprintf("kafka-message-data-%d", time.Now().UnixNano())
return key, []byte(value)
}
recordOffset += varintLen
var key []byte
if keyLength == -1 {
key = nil // null key
fmt.Printf("DEBUG: Record has null key\n")
} else if keyLength == 0 {
key = []byte{} // empty key
fmt.Printf("DEBUG: Record has empty key\n")
} else {
if recordOffset+int(keyLength) > len(recordData) {
fmt.Printf("DEBUG: Key extends beyond record, returning placeholder\n")
key = []byte("kafka-key")
value := fmt.Sprintf("kafka-message-data-%d", time.Now().UnixNano())
return key, []byte(value)
}
key = recordData[recordOffset : recordOffset+int(keyLength)]
recordOffset += int(keyLength)
fmt.Printf("DEBUG: Extracted key: %s\n", string(key))
}
// Read value length and value
valueLength, varintLen := decodeVarint(recordData[recordOffset:])
if varintLen == 0 {
fmt.Printf("DEBUG: Failed to decode value length, returning placeholder\n")
if key == nil {
key = []byte("kafka-key")
}
value := fmt.Sprintf("kafka-message-data-%d", time.Now().UnixNano())
return key, []byte(value)
}
recordOffset += varintLen
var value []byte
if valueLength == -1 {
value = nil // null value
fmt.Printf("DEBUG: Record has null value\n")
} else if valueLength == 0 {
value = []byte{} // empty value
fmt.Printf("DEBUG: Record has empty value\n")
} else {
if recordOffset+int(valueLength) > len(recordData) {
fmt.Printf("DEBUG: Value extends beyond record, returning placeholder\n")
if key == nil {
key = []byte("kafka-key")
}
value := fmt.Sprintf("kafka-message-data-%d", time.Now().UnixNano())
return key, []byte(value)
}
value = recordData[recordOffset : recordOffset+int(valueLength)]
fmt.Printf("DEBUG: Extracted value: %s\n", string(value))
}
if key == nil {
key = []byte{} // convert null key to empty for consistency
}
if value == nil {
value = []byte{} // convert null value to empty for consistency
}
fmt.Printf("DEBUG: Successfully extracted record - key: %s, value: %s\n", string(key), string(value))
return key, value
}
// decodeVarint decodes a variable-length integer from bytes
// Returns the decoded value and the number of bytes consumed
func decodeVarint(data []byte) (int64, int) {
if len(data) == 0 {
return 0, 0
}
var result int64
var shift uint
var bytesRead int
for i, b := range data {
if i > 9 { // varints can be at most 10 bytes
return 0, 0 // invalid varint
}
bytesRead++
result |= int64(b&0x7F) << shift
if (b & 0x80) == 0 {
// Most significant bit is 0, we're done
// Apply zigzag decoding for signed integers
return (result >> 1) ^ (-(result & 1)), bytesRead
}
shift += 7
}
return 0, 0 // incomplete varint
}
// handleProduceV2Plus handles Produce API v2-v7 (Kafka 0.11+)
func (h *Handler) handleProduceV2Plus(correlationID uint32, apiVersion uint16, requestBody []byte) ([]byte, error) {

Loading…
Cancel
Save