|
|
@ -3,6 +3,7 @@ package integration |
|
|
|
import ( |
|
|
|
"context" |
|
|
|
"fmt" |
|
|
|
"sync" |
|
|
|
"time" |
|
|
|
|
|
|
|
"github.com/seaweedfs/seaweedfs/weed/glog" |
|
|
@ -145,14 +146,46 @@ func (bc *BrokerClient) getOrCreatePublisher(topic string, partition int32) (*Br |
|
|
|
} |
|
|
|
bc.publishersLock.RUnlock() |
|
|
|
|
|
|
|
// Create new publisher stream
|
|
|
|
bc.publishersLock.Lock() |
|
|
|
defer bc.publishersLock.Unlock() |
|
|
|
// CRITICAL FIX: Prevent multiple concurrent attempts to create the same publisher
|
|
|
|
// Use a creation lock that is specific to each topic-partition pair
|
|
|
|
// This ensures only ONE goroutine tries to create/initialize for each publisher
|
|
|
|
if bc.publisherCreationLocks == nil { |
|
|
|
bc.publishersLock.Lock() |
|
|
|
if bc.publisherCreationLocks == nil { |
|
|
|
bc.publisherCreationLocks = make(map[string]*sync.Mutex) |
|
|
|
} |
|
|
|
bc.publishersLock.Unlock() |
|
|
|
} |
|
|
|
|
|
|
|
bc.publishersLock.RLock() |
|
|
|
creationLock, exists := bc.publisherCreationLocks[key] |
|
|
|
if !exists { |
|
|
|
// Need to create a creation lock for this topic-partition
|
|
|
|
bc.publishersLock.RUnlock() |
|
|
|
bc.publishersLock.Lock() |
|
|
|
// Double-check if someone else created it
|
|
|
|
if lock, exists := bc.publisherCreationLocks[key]; exists { |
|
|
|
creationLock = lock |
|
|
|
} else { |
|
|
|
creationLock = &sync.Mutex{} |
|
|
|
bc.publisherCreationLocks[key] = creationLock |
|
|
|
} |
|
|
|
bc.publishersLock.Unlock() |
|
|
|
} else { |
|
|
|
bc.publishersLock.RUnlock() |
|
|
|
} |
|
|
|
|
|
|
|
// Double-check after acquiring write lock
|
|
|
|
// Acquire the creation lock - only ONE goroutine will proceed
|
|
|
|
creationLock.Lock() |
|
|
|
defer creationLock.Unlock() |
|
|
|
|
|
|
|
// Double-check if publisher was created while we were waiting for the lock
|
|
|
|
bc.publishersLock.RLock() |
|
|
|
if session, exists := bc.publishers[key]; exists { |
|
|
|
bc.publishersLock.RUnlock() |
|
|
|
return session, nil |
|
|
|
} |
|
|
|
bc.publishersLock.RUnlock() |
|
|
|
|
|
|
|
// Create the stream
|
|
|
|
stream, err := bc.client.PublishMessage(bc.ctx) |
|
|
@ -160,13 +193,13 @@ func (bc *BrokerClient) getOrCreatePublisher(topic string, partition int32) (*Br |
|
|
|
return nil, fmt.Errorf("failed to create publish stream: %v", err) |
|
|
|
} |
|
|
|
|
|
|
|
// Get the actual partition assignment from the broker instead of using Kafka partition mapping
|
|
|
|
// Get the actual partition assignment from the broker
|
|
|
|
actualPartition, err := bc.getActualPartitionAssignment(topic, partition) |
|
|
|
if err != nil { |
|
|
|
return nil, fmt.Errorf("failed to get actual partition assignment: %v", err) |
|
|
|
} |
|
|
|
|
|
|
|
// Send init message using the actual partition structure that the broker allocated
|
|
|
|
// Send init message
|
|
|
|
if err := stream.Send(&mq_pb.PublishMessageRequest{ |
|
|
|
Message: &mq_pb.PublishMessageRequest_Init{ |
|
|
|
Init: &mq_pb.PublishMessageRequest_InitMessage{ |
|
|
@ -183,9 +216,7 @@ func (bc *BrokerClient) getOrCreatePublisher(topic string, partition int32) (*Br |
|
|
|
return nil, fmt.Errorf("failed to send init message: %v", err) |
|
|
|
} |
|
|
|
|
|
|
|
// CRITICAL: Consume the "hello" message sent by broker after init
|
|
|
|
// Broker sends empty PublishMessageResponse{} on line 137 of broker_grpc_pub.go
|
|
|
|
// Without this, first Recv() in PublishRecord gets hello instead of data ack
|
|
|
|
// Consume the "hello" message sent by broker after init
|
|
|
|
helloResp, err := stream.Recv() |
|
|
|
if err != nil { |
|
|
|
return nil, fmt.Errorf("failed to receive hello message: %v", err) |
|
|
@ -200,7 +231,11 @@ func (bc *BrokerClient) getOrCreatePublisher(topic string, partition int32) (*Br |
|
|
|
Stream: stream, |
|
|
|
} |
|
|
|
|
|
|
|
// Store in the map under the publishersLock
|
|
|
|
bc.publishersLock.Lock() |
|
|
|
bc.publishers[key] = session |
|
|
|
bc.publishersLock.Unlock() |
|
|
|
|
|
|
|
return session, nil |
|
|
|
} |
|
|
|
|
|
|
|