mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2025-10-08 03:14:24 +08:00
reduce logs, reduce CPU usages
This commit is contained in:
@@ -3,13 +3,14 @@ package cluster
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/seaweedfs/seaweedfs/weed/cluster/lock_manager"
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||
"google.golang.org/grpc"
|
||||
"time"
|
||||
)
|
||||
|
||||
type LockClient struct {
|
||||
@@ -109,7 +110,7 @@ func (lock *LiveLock) retryUntilLocked(lockDuration time.Duration) {
|
||||
}
|
||||
|
||||
func (lock *LiveLock) AttemptToLock(lockDuration time.Duration) error {
|
||||
glog.V(0).Infof("🔥 LOCK: AttemptToLock key=%s owner=%s", lock.key, lock.self)
|
||||
glog.V(4).Infof("LOCK: AttemptToLock key=%s owner=%s", lock.key, lock.self)
|
||||
errorMessage, err := lock.doLock(lockDuration)
|
||||
if err != nil {
|
||||
glog.V(0).Infof("❌ LOCK: doLock failed for key=%s: %v", lock.key, err)
|
||||
@@ -121,8 +122,11 @@ func (lock *LiveLock) AttemptToLock(lockDuration time.Duration) error {
|
||||
time.Sleep(time.Second)
|
||||
return fmt.Errorf("%v", errorMessage)
|
||||
}
|
||||
if !lock.isLocked {
|
||||
// Only log when transitioning from unlocked to locked
|
||||
glog.V(0).Infof("✅ LOCK: Successfully acquired key=%s owner=%s", lock.key, lock.self)
|
||||
}
|
||||
lock.isLocked = true
|
||||
glog.V(0).Infof("✅ LOCK: Successfully locked key=%s owner=%s", lock.key, lock.self)
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -151,14 +155,18 @@ func (lock *LiveLock) Stop() error {
|
||||
default:
|
||||
close(lock.cancelCh)
|
||||
}
|
||||
|
||||
|
||||
// Also release the lock if held
|
||||
return lock.StopShortLivedLock()
|
||||
}
|
||||
|
||||
func (lock *LiveLock) doLock(lockDuration time.Duration) (errorMessage string, err error) {
|
||||
glog.V(0).Infof("🔥 LOCK: doLock calling DistributedLock - key=%s filer=%s owner=%s renewToken=%s",
|
||||
lock.key, lock.hostFiler, lock.self, lock.renewToken)
|
||||
glog.V(4).Infof("LOCK: doLock calling DistributedLock - key=%s filer=%s owner=%s",
|
||||
lock.key, lock.hostFiler, lock.self)
|
||||
|
||||
previousHostFiler := lock.hostFiler
|
||||
previousOwner := lock.owner
|
||||
|
||||
err = pb.WithFilerClient(false, 0, lock.hostFiler, lock.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
|
||||
resp, err := client.DistributedLock(context.Background(), &filer_pb.LockRequest{
|
||||
Name: lock.key,
|
||||
@@ -167,28 +175,34 @@ func (lock *LiveLock) doLock(lockDuration time.Duration) (errorMessage string, e
|
||||
IsMoved: false,
|
||||
Owner: lock.self,
|
||||
})
|
||||
glog.V(0).Infof("🔍 LOCK: DistributedLock response - key=%s err=%v resp=%+v", lock.key, err, resp)
|
||||
glog.V(4).Infof("LOCK: DistributedLock response - key=%s err=%v", lock.key, err)
|
||||
if err == nil && resp != nil {
|
||||
lock.renewToken = resp.RenewToken
|
||||
glog.V(0).Infof("✅ LOCK: Got renewToken for key=%s: %s", lock.key, lock.renewToken)
|
||||
glog.V(4).Infof("LOCK: Got renewToken for key=%s", lock.key)
|
||||
} else {
|
||||
//this can be retried. Need to remember the last valid renewToken
|
||||
lock.renewToken = ""
|
||||
glog.V(0).Infof("⚠️ LOCK: Cleared renewToken for key=%s", lock.key)
|
||||
glog.V(0).Infof("⚠️ LOCK: Cleared renewToken for key=%s (err=%v)", lock.key, err)
|
||||
}
|
||||
if resp != nil {
|
||||
errorMessage = resp.Error
|
||||
if resp.LockHostMovedTo != "" {
|
||||
glog.V(0).Infof("🔄 LOCK: Lock moved to %s for key=%s", resp.LockHostMovedTo, lock.key)
|
||||
if resp.LockHostMovedTo != "" && resp.LockHostMovedTo != string(previousHostFiler) {
|
||||
// Only log if the host actually changed
|
||||
glog.V(0).Infof("🔄 LOCK: Host changed from %s to %s for key=%s", previousHostFiler, resp.LockHostMovedTo, lock.key)
|
||||
lock.hostFiler = pb.ServerAddress(resp.LockHostMovedTo)
|
||||
lock.lc.seedFiler = lock.hostFiler
|
||||
} else if resp.LockHostMovedTo != "" {
|
||||
lock.hostFiler = pb.ServerAddress(resp.LockHostMovedTo)
|
||||
}
|
||||
if resp.LockOwner != "" {
|
||||
if resp.LockOwner != "" && resp.LockOwner != previousOwner {
|
||||
// Only log if the owner actually changed
|
||||
glog.V(0).Infof("👤 LOCK: Owner changed from %s to %s for key=%s", previousOwner, resp.LockOwner, lock.key)
|
||||
lock.owner = resp.LockOwner
|
||||
glog.V(0).Infof("👤 LOCK: Lock owner is %s for key=%s", lock.owner, lock.key)
|
||||
} else {
|
||||
} else if resp.LockOwner != "" {
|
||||
lock.owner = resp.LockOwner
|
||||
} else if previousOwner != "" {
|
||||
glog.V(0).Infof("⚠️ LOCK: Owner cleared for key=%s", lock.key)
|
||||
lock.owner = ""
|
||||
glog.V(0).Infof("⚠️ LOCK: No lock owner for key=%s", lock.key)
|
||||
}
|
||||
}
|
||||
return err
|
||||
|
@@ -2,6 +2,8 @@ package command
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"net/http"
|
||||
_ "net/http/pprof"
|
||||
"os"
|
||||
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
@@ -105,6 +107,17 @@ func runMqKafkaGateway(cmd *Command, args []string) bool {
|
||||
}
|
||||
glog.V(0).Infof("Using SeaweedMQ brokers from masters: %s", *mqKafkaGatewayOptions.master)
|
||||
|
||||
// Start HTTP profiling server
|
||||
pprofPort := *mqKafkaGatewayOptions.port + 1000 // e.g., 10093 for profiling if gateway is on 9093
|
||||
go func() {
|
||||
pprofAddr := fmt.Sprintf(":%d", pprofPort)
|
||||
glog.V(0).Infof("Kafka Gateway pprof server listening on %s", pprofAddr)
|
||||
glog.V(0).Infof("Access profiling at: http://localhost:%d/debug/pprof/", pprofPort)
|
||||
if err := http.ListenAndServe(pprofAddr, nil); err != nil {
|
||||
glog.Errorf("pprof server error: %v", err)
|
||||
}
|
||||
}()
|
||||
|
||||
if err := srv.Start(); err != nil {
|
||||
glog.Fatalf("mq kafka gateway start: %v", err)
|
||||
return false
|
||||
|
@@ -38,13 +38,13 @@ func (b *MessageQueueBroker) SubscribeMessage(stream mq_pb.SeaweedMessaging_Subs
|
||||
|
||||
glog.V(0).Infof("Subscriber %s on %v %v connected", req.GetInit().ConsumerId, t, partition)
|
||||
|
||||
glog.V(0).Infof("🔍 DEBUG: Calling GetOrGenerateLocalPartition for %s %s", t, partition)
|
||||
glog.V(4).Infof("Calling GetOrGenerateLocalPartition for %s %s", t, partition)
|
||||
localTopicPartition, getOrGenErr := b.GetOrGenerateLocalPartition(t, partition)
|
||||
if getOrGenErr != nil {
|
||||
glog.V(0).Infof("🔍 DEBUG: GetOrGenerateLocalPartition failed: %v", getOrGenErr)
|
||||
glog.V(4).Infof("GetOrGenerateLocalPartition failed: %v", getOrGenErr)
|
||||
return getOrGenErr
|
||||
}
|
||||
glog.V(0).Infof("🔍 DEBUG: GetOrGenerateLocalPartition succeeded, localTopicPartition=%v", localTopicPartition != nil)
|
||||
glog.V(4).Infof("GetOrGenerateLocalPartition succeeded, localTopicPartition=%v", localTopicPartition != nil)
|
||||
if localTopicPartition == nil {
|
||||
return fmt.Errorf("failed to get or generate local partition for topic %v partition %v", t, partition)
|
||||
}
|
||||
|
@@ -46,18 +46,18 @@ func (b *MessageQueueBroker) doGetOrGenLocalPartition(t topic.Topic, partition t
|
||||
|
||||
func (b *MessageQueueBroker) genLocalPartitionFromFiler(t topic.Topic, partition topic.Partition, conf *mq_pb.ConfigureTopicResponse) (localPartition *topic.LocalPartition, isGenerated bool, err error) {
|
||||
self := b.option.BrokerAddress()
|
||||
glog.V(0).Infof("🔍 DEBUG: genLocalPartitionFromFiler for %s %s, self=%s", t, partition, self)
|
||||
glog.V(0).Infof("🔍 DEBUG: conf.BrokerPartitionAssignments: %v", conf.BrokerPartitionAssignments)
|
||||
glog.V(4).Infof("genLocalPartitionFromFiler for %s %s, self=%s", t, partition, self)
|
||||
glog.V(4).Infof("conf.BrokerPartitionAssignments: %v", conf.BrokerPartitionAssignments)
|
||||
for _, assignment := range conf.BrokerPartitionAssignments {
|
||||
assignmentPartition := topic.FromPbPartition(assignment.Partition)
|
||||
glog.V(0).Infof("🔍 DEBUG: checking assignment: LeaderBroker=%s, Partition=%s", assignment.LeaderBroker, assignmentPartition)
|
||||
glog.V(0).Infof("🔍 DEBUG: comparing self=%s with LeaderBroker=%s: %v", self, assignment.LeaderBroker, assignment.LeaderBroker == string(self))
|
||||
glog.V(0).Infof("🔍 DEBUG: comparing partition=%s with assignmentPartition=%s: %v", partition.String(), assignmentPartition.String(), partition.Equals(assignmentPartition))
|
||||
glog.V(0).Infof("🔍 DEBUG: logical comparison (RangeStart, RangeStop only): %v", partition.LogicalEquals(assignmentPartition))
|
||||
glog.V(0).Infof("🔍 DEBUG: partition details: RangeStart=%d, RangeStop=%d, RingSize=%d, UnixTimeNs=%d", partition.RangeStart, partition.RangeStop, partition.RingSize, partition.UnixTimeNs)
|
||||
glog.V(0).Infof("🔍 DEBUG: assignmentPartition details: RangeStart=%d, RangeStop=%d, RingSize=%d, UnixTimeNs=%d", assignmentPartition.RangeStart, assignmentPartition.RangeStop, assignmentPartition.RingSize, assignmentPartition.UnixTimeNs)
|
||||
glog.V(4).Infof("checking assignment: LeaderBroker=%s, Partition=%s", assignment.LeaderBroker, assignmentPartition)
|
||||
glog.V(4).Infof("comparing self=%s with LeaderBroker=%s: %v", self, assignment.LeaderBroker, assignment.LeaderBroker == string(self))
|
||||
glog.V(4).Infof("comparing partition=%s with assignmentPartition=%s: %v", partition.String(), assignmentPartition.String(), partition.Equals(assignmentPartition))
|
||||
glog.V(4).Infof("logical comparison (RangeStart, RangeStop only): %v", partition.LogicalEquals(assignmentPartition))
|
||||
glog.V(4).Infof("partition details: RangeStart=%d, RangeStop=%d, RingSize=%d, UnixTimeNs=%d", partition.RangeStart, partition.RangeStop, partition.RingSize, partition.UnixTimeNs)
|
||||
glog.V(4).Infof("assignmentPartition details: RangeStart=%d, RangeStop=%d, RingSize=%d, UnixTimeNs=%d", assignmentPartition.RangeStart, assignmentPartition.RangeStop, assignmentPartition.RingSize, assignmentPartition.UnixTimeNs)
|
||||
if assignment.LeaderBroker == string(self) && partition.LogicalEquals(assignmentPartition) {
|
||||
glog.V(0).Infof("🔍 DEBUG: Creating local partition for %s %s", t, partition)
|
||||
glog.V(4).Infof("Creating local partition for %s %s", t, partition)
|
||||
localPartition = topic.NewLocalPartition(partition, b.option.LogFlushInterval, b.genLogFlushFunc(t, partition), logstore.GenMergedReadFunc(b, t, partition))
|
||||
|
||||
// Initialize offset from existing data to ensure continuity on restart
|
||||
@@ -65,13 +65,13 @@ func (b *MessageQueueBroker) genLocalPartitionFromFiler(t topic.Topic, partition
|
||||
|
||||
b.localTopicManager.AddLocalPartition(t, localPartition)
|
||||
isGenerated = true
|
||||
glog.V(0).Infof("🔍 DEBUG: Successfully added local partition %s %s to localTopicManager", t, partition)
|
||||
glog.V(4).Infof("Successfully added local partition %s %s to localTopicManager", t, partition)
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if !isGenerated {
|
||||
glog.V(0).Infof("🔍 DEBUG: No matching assignment found for %s %s", t, partition)
|
||||
glog.V(4).Infof("No matching assignment found for %s %s", t, partition)
|
||||
}
|
||||
|
||||
return localPartition, isGenerated, nil
|
||||
|
@@ -177,11 +177,9 @@ func (s *Server) Start() error {
|
||||
s.wg.Add(1)
|
||||
go func(c net.Conn) {
|
||||
defer s.wg.Done()
|
||||
fmt.Printf("🔥 GATEWAY DEBUG: About to call handler.HandleConn for %s\n", c.RemoteAddr())
|
||||
if err := s.handler.HandleConn(s.ctx, c); err != nil {
|
||||
glog.V(1).Infof("handle conn %v: %v", c.RemoteAddr(), err)
|
||||
}
|
||||
fmt.Printf("🔥 GATEWAY DEBUG: handler.HandleConn completed for %s\n", c.RemoteAddr())
|
||||
}(conn)
|
||||
}
|
||||
}()
|
||||
|
@@ -41,6 +41,12 @@ type hwmCacheEntry struct {
|
||||
expiresAt time.Time
|
||||
}
|
||||
|
||||
// topicExistsCacheEntry represents a cached topic existence check
|
||||
type topicExistsCacheEntry struct {
|
||||
exists bool
|
||||
expiresAt time.Time
|
||||
}
|
||||
|
||||
// SeaweedMQHandler integrates Kafka protocol handlers with real SeaweedMQ storage
|
||||
type SeaweedMQHandler struct {
|
||||
// Shared filer client accessor for all components
|
||||
@@ -67,6 +73,11 @@ type SeaweedMQHandler struct {
|
||||
hwmCache map[string]*hwmCacheEntry // key: "topic:partition"
|
||||
hwmCacheMu sync.RWMutex
|
||||
hwmCacheTTL time.Duration
|
||||
|
||||
// Topic existence cache to reduce broker queries
|
||||
topicExistsCache map[string]*topicExistsCacheEntry // key: "topic"
|
||||
topicExistsCacheMu sync.RWMutex
|
||||
topicExistsCacheTTL time.Duration
|
||||
}
|
||||
|
||||
// ConnectionContext holds connection-specific information for requests
|
||||
@@ -256,23 +267,23 @@ type PartitionRangeInfo struct {
|
||||
// GetEarliestOffset returns the earliest available offset for a topic partition
|
||||
// ALWAYS queries SMQ broker directly - no ledger involved
|
||||
func (h *SeaweedMQHandler) GetEarliestOffset(topic string, partition int32) (int64, error) {
|
||||
glog.Infof("[DEBUG_OFFSET] GetEarliestOffset called for topic=%s partition=%d", topic, partition)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] GetEarliestOffset called for topic=%s partition=%d", topic, partition)
|
||||
|
||||
// Check if topic exists
|
||||
if !h.TopicExists(topic) {
|
||||
glog.Infof("[DEBUG_OFFSET] Topic %s does not exist", topic)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Topic %s does not exist", topic)
|
||||
return 0, nil // Empty topic starts at offset 0
|
||||
}
|
||||
|
||||
// ALWAYS query SMQ broker directly for earliest offset
|
||||
if h.brokerClient != nil {
|
||||
glog.Infof("[DEBUG_OFFSET] Querying SMQ broker for earliest offset...")
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Querying SMQ broker for earliest offset...")
|
||||
earliestOffset, err := h.brokerClient.GetEarliestOffset(topic, partition)
|
||||
if err != nil {
|
||||
glog.Errorf("[DEBUG_OFFSET] Failed to get earliest offset from broker: %v", err)
|
||||
return 0, err
|
||||
}
|
||||
glog.Infof("[DEBUG_OFFSET] Got earliest offset from broker: %d", earliestOffset)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Got earliest offset from broker: %d", earliestOffset)
|
||||
return earliestOffset, nil
|
||||
}
|
||||
|
||||
@@ -399,7 +410,7 @@ func (h *SeaweedMQHandler) CreatePerConnectionBrokerClient() (*BrokerClient, err
|
||||
|
||||
// Use the first broker address (in production, could use load balancing)
|
||||
brokerAddress := h.brokerAddresses[0]
|
||||
glog.Infof("[BROKER_CLIENT] Creating per-connection client to %s", brokerAddress)
|
||||
glog.V(4).Infof("[BROKER_CLIENT] Creating per-connection client to %s", brokerAddress)
|
||||
|
||||
// Create a new client with the shared filer accessor
|
||||
client, err := NewBrokerClientWithFilerAccessor(brokerAddress, h.filerClientAccessor)
|
||||
@@ -407,7 +418,7 @@ func (h *SeaweedMQHandler) CreatePerConnectionBrokerClient() (*BrokerClient, err
|
||||
return nil, fmt.Errorf("failed to create broker client: %w", err)
|
||||
}
|
||||
|
||||
glog.Infof("[BROKER_CLIENT] Successfully created per-connection client")
|
||||
glog.V(4).Infof("[BROKER_CLIENT] Successfully created per-connection client")
|
||||
return client, nil
|
||||
}
|
||||
|
||||
@@ -478,6 +489,9 @@ func (h *SeaweedMQHandler) CreateTopicWithSchemas(name string, partitions int32,
|
||||
|
||||
// Offset management now handled directly by SMQ broker - no initialization needed
|
||||
|
||||
// Invalidate cache after successful topic creation
|
||||
h.InvalidateTopicExistsCache(name)
|
||||
|
||||
glog.V(1).Infof("Topic %s created successfully with %d partitions", name, partitions)
|
||||
return nil
|
||||
}
|
||||
@@ -565,19 +579,56 @@ func (h *SeaweedMQHandler) DeleteTopic(name string) error {
|
||||
}
|
||||
|
||||
// TopicExists checks if a topic exists in SeaweedMQ broker (includes in-memory topics)
|
||||
// Uses a 5-second cache to reduce broker queries
|
||||
func (h *SeaweedMQHandler) TopicExists(name string) bool {
|
||||
// Check cache first
|
||||
h.topicExistsCacheMu.RLock()
|
||||
if entry, found := h.topicExistsCache[name]; found {
|
||||
if time.Now().Before(entry.expiresAt) {
|
||||
h.topicExistsCacheMu.RUnlock()
|
||||
glog.V(4).Infof("TopicExists cache HIT for %s: %v", name, entry.exists)
|
||||
return entry.exists
|
||||
}
|
||||
}
|
||||
h.topicExistsCacheMu.RUnlock()
|
||||
|
||||
// Cache miss or expired - query broker
|
||||
glog.V(4).Infof("TopicExists cache MISS for %s, querying broker", name)
|
||||
|
||||
var exists bool
|
||||
// Check via SeaweedMQ broker (includes in-memory topics)
|
||||
if h.brokerClient != nil {
|
||||
exists, err := h.brokerClient.TopicExists(name)
|
||||
if err == nil {
|
||||
return exists
|
||||
var err error
|
||||
exists, err = h.brokerClient.TopicExists(name)
|
||||
if err != nil {
|
||||
fmt.Printf("TopicExists: Failed to check topic %s via SMQ broker: %v\n", name, err)
|
||||
// Don't cache errors
|
||||
return false
|
||||
}
|
||||
fmt.Printf("TopicExists: Failed to check topic %s via SMQ broker: %v\n", name, err)
|
||||
} else {
|
||||
// Return false if broker is unavailable
|
||||
fmt.Printf("TopicExists: No broker client available for topic %s\n", name)
|
||||
return false
|
||||
}
|
||||
|
||||
// Return false if broker is unavailable
|
||||
fmt.Printf("TopicExists: No broker client available for topic %s\n", name)
|
||||
return false
|
||||
// Update cache
|
||||
h.topicExistsCacheMu.Lock()
|
||||
h.topicExistsCache[name] = &topicExistsCacheEntry{
|
||||
exists: exists,
|
||||
expiresAt: time.Now().Add(h.topicExistsCacheTTL),
|
||||
}
|
||||
h.topicExistsCacheMu.Unlock()
|
||||
|
||||
return exists
|
||||
}
|
||||
|
||||
// InvalidateTopicExistsCache removes a topic from the existence cache
|
||||
// Should be called after creating or deleting a topic
|
||||
func (h *SeaweedMQHandler) InvalidateTopicExistsCache(name string) {
|
||||
h.topicExistsCacheMu.Lock()
|
||||
delete(h.topicExistsCache, name)
|
||||
h.topicExistsCacheMu.Unlock()
|
||||
glog.V(4).Infof("Invalidated TopicExists cache for %s", name)
|
||||
}
|
||||
|
||||
// GetTopicInfo returns information about a topic from filer
|
||||
@@ -1004,9 +1055,11 @@ func NewSeaweedMQBrokerHandler(masters string, filerGroup string, clientHost str
|
||||
masterClient: masterClient,
|
||||
// topics map removed - always read from filer directly
|
||||
// ledgers removed - SMQ broker handles all offset management
|
||||
brokerAddresses: brokerAddresses, // Store all discovered broker addresses
|
||||
hwmCache: make(map[string]*hwmCacheEntry),
|
||||
hwmCacheTTL: 2 * time.Second, // 2 second cache TTL to reduce broker queries
|
||||
brokerAddresses: brokerAddresses, // Store all discovered broker addresses
|
||||
hwmCache: make(map[string]*hwmCacheEntry),
|
||||
hwmCacheTTL: 2 * time.Second, // 2 second cache TTL to reduce broker queries
|
||||
topicExistsCache: make(map[string]*topicExistsCacheEntry),
|
||||
topicExistsCacheTTL: 5 * time.Second, // 5 second cache TTL for topic existence
|
||||
}, nil
|
||||
}
|
||||
|
||||
@@ -1254,14 +1307,12 @@ func NewBrokerClientWithFilerAccessor(brokerAddress string, filerClientAccessor
|
||||
|
||||
// Close shuts down the broker client and all streams
|
||||
func (bc *BrokerClient) Close() error {
|
||||
glog.V(0).Infof("🔍 BrokerClient.Close() called - closing %d publishers and %d subscribers", len(bc.publishers), len(bc.subscribers))
|
||||
bc.cancel()
|
||||
|
||||
// Close all publisher streams
|
||||
bc.publishersLock.Lock()
|
||||
for key, session := range bc.publishers {
|
||||
if session.Stream != nil {
|
||||
glog.V(0).Infof("🔍 Closing publisher stream for %s", key)
|
||||
_ = session.Stream.CloseSend()
|
||||
}
|
||||
delete(bc.publishers, key)
|
||||
@@ -1271,7 +1322,6 @@ func (bc *BrokerClient) Close() error {
|
||||
// Close all subscriber streams
|
||||
bc.subscribersLock.Lock()
|
||||
for key, session := range bc.subscribers {
|
||||
glog.V(0).Infof("🔍 Closing subscriber stream for %s", key)
|
||||
if session.Stream != nil {
|
||||
_ = session.Stream.CloseSend()
|
||||
}
|
||||
@@ -1282,13 +1332,12 @@ func (bc *BrokerClient) Close() error {
|
||||
}
|
||||
bc.subscribersLock.Unlock()
|
||||
|
||||
glog.V(0).Infof("🔍 BrokerClient.Close() complete, closing gRPC connection")
|
||||
return bc.conn.Close()
|
||||
}
|
||||
|
||||
// GetPartitionRangeInfo gets comprehensive range information from SeaweedMQ broker's native range manager
|
||||
func (bc *BrokerClient) GetPartitionRangeInfo(topic string, partition int32) (*PartitionRangeInfo, error) {
|
||||
glog.Infof("[DEBUG_OFFSET] GetPartitionRangeInfo called for topic=%s partition=%d", topic, partition)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] GetPartitionRangeInfo called for topic=%s partition=%d", topic, partition)
|
||||
|
||||
if bc.client == nil {
|
||||
return nil, fmt.Errorf("broker client not connected")
|
||||
@@ -1312,12 +1361,12 @@ func (bc *BrokerClient) GetPartitionRangeInfo(topic string, partition int32) (*P
|
||||
Partition: actualPartition,
|
||||
})
|
||||
if err != nil {
|
||||
glog.Infof("[DEBUG_OFFSET] Failed to call GetPartitionRangeInfo gRPC: %v", err)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Failed to call GetPartitionRangeInfo gRPC: %v", err)
|
||||
return nil, fmt.Errorf("failed to get partition range info from broker: %v", err)
|
||||
}
|
||||
|
||||
if resp.Error != "" {
|
||||
glog.Infof("[DEBUG_OFFSET] Broker returned error: %s", resp.Error)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Broker returned error: %s", resp.Error)
|
||||
return nil, fmt.Errorf("broker error: %s", resp.Error)
|
||||
}
|
||||
|
||||
@@ -1346,7 +1395,7 @@ func (bc *BrokerClient) GetPartitionRangeInfo(topic string, partition int32) (*P
|
||||
ActiveSubscriptions: resp.ActiveSubscriptions,
|
||||
}
|
||||
|
||||
glog.Infof("[DEBUG_OFFSET] Got range info from broker: earliest=%d, latest=%d, hwm=%d, records=%d, ts_range=[%d,%d]",
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Got range info from broker: earliest=%d, latest=%d, hwm=%d, records=%d, ts_range=[%d,%d]",
|
||||
info.EarliestOffset, info.LatestOffset, info.HighWaterMark, info.RecordCount,
|
||||
info.EarliestTimestampNs, info.LatestTimestampNs)
|
||||
return info, nil
|
||||
@@ -1354,23 +1403,23 @@ func (bc *BrokerClient) GetPartitionRangeInfo(topic string, partition int32) (*P
|
||||
|
||||
// GetHighWaterMark gets the high water mark for a topic partition
|
||||
func (bc *BrokerClient) GetHighWaterMark(topic string, partition int32) (int64, error) {
|
||||
glog.Infof("[DEBUG_OFFSET] GetHighWaterMark called for topic=%s partition=%d", topic, partition)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] GetHighWaterMark called for topic=%s partition=%d", topic, partition)
|
||||
|
||||
// Primary approach: Use SeaweedMQ's native range manager via gRPC
|
||||
info, err := bc.GetPartitionRangeInfo(topic, partition)
|
||||
if err != nil {
|
||||
glog.Infof("[DEBUG_OFFSET] Failed to get offset info from broker, falling back to chunk metadata: %v", err)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Failed to get offset info from broker, falling back to chunk metadata: %v", err)
|
||||
// Fallback to chunk metadata approach
|
||||
highWaterMark, err := bc.getHighWaterMarkFromChunkMetadata(topic, partition)
|
||||
if err != nil {
|
||||
glog.Infof("[DEBUG_OFFSET] Failed to get high water mark from chunk metadata: %v", err)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Failed to get high water mark from chunk metadata: %v", err)
|
||||
return 0, err
|
||||
}
|
||||
glog.Infof("[DEBUG_OFFSET] Got high water mark from chunk metadata fallback: %d", highWaterMark)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Got high water mark from chunk metadata fallback: %d", highWaterMark)
|
||||
return highWaterMark, nil
|
||||
}
|
||||
|
||||
glog.Infof("[DEBUG_OFFSET] Successfully got high water mark from broker: %d", info.HighWaterMark)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Successfully got high water mark from broker: %d", info.HighWaterMark)
|
||||
return info.HighWaterMark, nil
|
||||
}
|
||||
|
||||
@@ -1414,7 +1463,7 @@ func (bc *BrokerClient) getOffsetRangeFromChunkMetadata(topic string, partition
|
||||
}
|
||||
|
||||
if latestVersion == "" {
|
||||
glog.Infof("[DEBUG_OFFSET] No version directory found for topic %s", topic)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] No version directory found for topic %s", topic)
|
||||
return 0, 0, nil
|
||||
}
|
||||
|
||||
@@ -1449,7 +1498,7 @@ func (bc *BrokerClient) getOffsetRangeFromChunkMetadata(topic string, partition
|
||||
}
|
||||
|
||||
if partitionDir == "" {
|
||||
glog.Infof("[DEBUG_OFFSET] No partition directory found for topic %s", topic)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] No partition directory found for topic %s", topic)
|
||||
return 0, 0, nil
|
||||
}
|
||||
|
||||
@@ -1488,7 +1537,7 @@ func (bc *BrokerClient) getOffsetRangeFromChunkMetadata(topic string, partition
|
||||
if maxOffset > highWaterMark {
|
||||
highWaterMark = maxOffset
|
||||
}
|
||||
glog.Infof("[DEBUG_OFFSET] %s file %s has offset_max=%d", fileType, resp.Entry.Name, maxOffset)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] %s file %s has offset_max=%d", fileType, resp.Entry.Name, maxOffset)
|
||||
}
|
||||
|
||||
// Track minimum offset for earliest offset
|
||||
@@ -1497,7 +1546,7 @@ func (bc *BrokerClient) getOffsetRangeFromChunkMetadata(topic string, partition
|
||||
if earliestOffset == -1 || minOffset < earliestOffset {
|
||||
earliestOffset = minOffset
|
||||
}
|
||||
glog.Infof("[DEBUG_OFFSET] %s file %s has offset_min=%d", fileType, resp.Entry.Name, minOffset)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] %s file %s has offset_min=%d", fileType, resp.Entry.Name, minOffset)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1518,7 +1567,7 @@ func (bc *BrokerClient) getOffsetRangeFromChunkMetadata(topic string, partition
|
||||
earliestOffset = 0
|
||||
}
|
||||
|
||||
glog.Infof("[DEBUG_OFFSET] Offset range for topic %s partition %d: earliest=%d, highWaterMark=%d", topic, partition, earliestOffset, highWaterMark)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Offset range for topic %s partition %d: earliest=%d, highWaterMark=%d", topic, partition, earliestOffset, highWaterMark)
|
||||
return earliestOffset, highWaterMark, nil
|
||||
}
|
||||
|
||||
@@ -1530,23 +1579,23 @@ func (bc *BrokerClient) getHighWaterMarkFromChunkMetadata(topic string, partitio
|
||||
|
||||
// GetEarliestOffset gets the earliest offset from SeaweedMQ broker's native offset manager
|
||||
func (bc *BrokerClient) GetEarliestOffset(topic string, partition int32) (int64, error) {
|
||||
glog.Infof("[DEBUG_OFFSET] BrokerClient.GetEarliestOffset called for topic=%s partition=%d", topic, partition)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] BrokerClient.GetEarliestOffset called for topic=%s partition=%d", topic, partition)
|
||||
|
||||
// Primary approach: Use SeaweedMQ's native range manager via gRPC
|
||||
info, err := bc.GetPartitionRangeInfo(topic, partition)
|
||||
if err != nil {
|
||||
glog.Infof("[DEBUG_OFFSET] Failed to get offset info from broker, falling back to chunk metadata: %v", err)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Failed to get offset info from broker, falling back to chunk metadata: %v", err)
|
||||
// Fallback to chunk metadata approach
|
||||
earliestOffset, err := bc.getEarliestOffsetFromChunkMetadata(topic, partition)
|
||||
if err != nil {
|
||||
glog.Infof("[DEBUG_OFFSET] Failed to get earliest offset from chunk metadata: %v", err)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Failed to get earliest offset from chunk metadata: %v", err)
|
||||
return 0, err
|
||||
}
|
||||
glog.Infof("[DEBUG_OFFSET] Got earliest offset from chunk metadata fallback: %d", earliestOffset)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Got earliest offset from chunk metadata fallback: %d", earliestOffset)
|
||||
return earliestOffset, nil
|
||||
}
|
||||
|
||||
glog.Infof("[DEBUG_OFFSET] Successfully got earliest offset from broker: %d", info.EarliestOffset)
|
||||
glog.V(4).Infof("[DEBUG_OFFSET] Successfully got earliest offset from broker: %d", info.EarliestOffset)
|
||||
return info.EarliestOffset, nil
|
||||
}
|
||||
|
||||
|
@@ -61,21 +61,33 @@ func (h *Handler) handleFetch(ctx context.Context, correlationID uint32, apiVers
|
||||
}
|
||||
return false
|
||||
}
|
||||
// Cap long-polling to avoid blocking connection shutdowns in tests
|
||||
// Long-poll when client requests it via MaxWaitTime and there's no data
|
||||
// Even if MinBytes=0, we should honor MaxWaitTime to reduce polling overhead
|
||||
maxWaitMs := fetchRequest.MaxWaitTime
|
||||
if maxWaitMs > 1000 {
|
||||
maxWaitMs = 1000
|
||||
// For production, allow longer wait times to reduce CPU usage
|
||||
// Only cap at 30 seconds to be reasonable
|
||||
if maxWaitMs > 30000 {
|
||||
maxWaitMs = 30000
|
||||
}
|
||||
shouldLongPoll := fetchRequest.MinBytes > 0 && maxWaitMs > 0 && !hasDataAvailable() && allTopicsExist()
|
||||
// Long-poll if: (1) client wants to wait (maxWaitMs > 0), (2) no data available, (3) topics exist
|
||||
// NOTE: We long-poll even if MinBytes=0, since the client specified a wait time
|
||||
hasData := hasDataAvailable()
|
||||
topicsExist := allTopicsExist()
|
||||
shouldLongPoll := maxWaitMs > 0 && !hasData && topicsExist
|
||||
// Debug Schema Registry polling (disabled for production)
|
||||
// Uncomment for debugging long-poll behavior
|
||||
/*
|
||||
if len(fetchRequest.Topics) > 0 && strings.HasPrefix(fetchRequest.Topics[0].Name, "_schemas") {
|
||||
glog.V(4).Infof("SR Fetch: maxWaitMs=%d minBytes=%d hasData=%v topicsExist=%v shouldLongPoll=%v",
|
||||
maxWaitMs, fetchRequest.MinBytes, hasData, topicsExist, shouldLongPoll)
|
||||
}
|
||||
*/
|
||||
if shouldLongPoll {
|
||||
start := time.Now()
|
||||
// Limit polling time to maximum 2 seconds to prevent hanging in CI
|
||||
// Use the client's requested wait time (already capped at 30s)
|
||||
maxPollTime := time.Duration(maxWaitMs) * time.Millisecond
|
||||
if maxPollTime > 2*time.Second {
|
||||
maxPollTime = 2 * time.Second
|
||||
Debug("Limiting fetch polling to 2 seconds to prevent hanging")
|
||||
}
|
||||
deadline := start.Add(maxPollTime)
|
||||
glog.V(4).Infof("Fetch long-polling: maxWaitMs=%d, deadline=%v", maxWaitMs, deadline)
|
||||
for time.Now().Before(deadline) {
|
||||
// Use context-aware sleep instead of blocking time.Sleep
|
||||
select {
|
||||
@@ -121,22 +133,22 @@ func (h *Handler) handleFetch(ctx context.Context, correlationID uint32, apiVers
|
||||
// Topics count - write the actual number of topics in the request
|
||||
// Kafka protocol: we MUST return all requested topics in the response (even with empty data)
|
||||
topicsCount := len(fetchRequest.Topics)
|
||||
glog.Infof("🔍 FETCH CORR=%d: Writing topics count=%d at offset=%d, isFlexible=%v", correlationID, topicsCount, len(response), isFlexible)
|
||||
glog.V(4).Infof("FETCH CORR=%d: Writing topics count=%d at offset=%d, isFlexible=%v", correlationID, topicsCount, len(response), isFlexible)
|
||||
if isFlexible {
|
||||
// Flexible versions use compact array format (count + 1)
|
||||
response = append(response, EncodeUvarint(uint32(topicsCount+1))...)
|
||||
} else {
|
||||
topicsCountBytes := make([]byte, 4)
|
||||
binary.BigEndian.PutUint32(topicsCountBytes, uint32(topicsCount))
|
||||
glog.Infof("🔍 FETCH CORR=%d: topicsCountBytes = %02x %02x %02x %02x", correlationID, topicsCountBytes[0], topicsCountBytes[1], topicsCountBytes[2], topicsCountBytes[3])
|
||||
glog.V(4).Infof("FETCH CORR=%d: topicsCountBytes = %02x %02x %02x %02x", correlationID, topicsCountBytes[0], topicsCountBytes[1], topicsCountBytes[2], topicsCountBytes[3])
|
||||
response = append(response, topicsCountBytes...)
|
||||
glog.Infof("🔍 FETCH CORR=%d: After appending topics count, response length=%d, response[10-13]=%02x %02x %02x %02x",
|
||||
glog.V(4).Infof("FETCH CORR=%d: After appending topics count, response length=%d, response[10-13]=%02x %02x %02x %02x",
|
||||
correlationID, len(response), response[10], response[11], response[12], response[13])
|
||||
}
|
||||
|
||||
// Process each requested topic
|
||||
for topicIdx, topic := range fetchRequest.Topics {
|
||||
glog.Infof("FETCH: Processing topic %d/%d: %s (partitions: %d)", topicIdx+1, len(fetchRequest.Topics), topic.Name, len(topic.Partitions))
|
||||
glog.V(4).Infof("FETCH: Processing topic %d/%d: %s (partitions: %d)", topicIdx+1, len(fetchRequest.Topics), topic.Name, len(topic.Partitions))
|
||||
topicNameBytes := []byte(topic.Name)
|
||||
|
||||
// Topic name length and name
|
||||
@@ -211,7 +223,7 @@ func (h *Handler) handleFetch(ctx context.Context, correlationID uint32, apiVers
|
||||
}
|
||||
|
||||
if strings.HasPrefix(topic.Name, "_schemas") {
|
||||
glog.Infof("📍 SR FETCH REQUEST: topic=%s partition=%d requestOffset=%d effectiveOffset=%d highWaterMark=%d",
|
||||
glog.V(4).Infof("SR FETCH REQUEST: topic=%s partition=%d requestOffset=%d effectiveOffset=%d highWaterMark=%d",
|
||||
topic.Name, partition.PartitionID, partition.FetchOffset, effectiveFetchOffset, highWaterMark)
|
||||
}
|
||||
|
||||
@@ -374,7 +386,7 @@ func (h *Handler) handleFetch(ctx context.Context, correlationID uint32, apiVers
|
||||
response = append(response, recordBatch...)
|
||||
totalAppendedRecordBytes += len(recordBatch)
|
||||
// Always log per-topic appended bytes for visibility
|
||||
glog.Infof("FETCH TOPIC APPENDED: corr=%d topic=%s partition=%d bytes=%d flexible=%v",
|
||||
glog.V(4).Infof("FETCH TOPIC APPENDED: corr=%d topic=%s partition=%d bytes=%d flexible=%v",
|
||||
correlationID, topic.Name, partition.PartitionID, len(recordBatch), isFlexible)
|
||||
|
||||
// Tagged fields for flexible versions (v12+) after each partition
|
||||
@@ -401,106 +413,32 @@ func (h *Handler) handleFetch(ctx context.Context, correlationID uint32, apiVers
|
||||
glog.Errorf("🚨 FETCH CORR=%d: Topics count CORRUPTED! Expected %d, found %d at response[10:14]=%02x %02x %02x %02x",
|
||||
correlationID, topicsCount, actualTopicsCount, response[10], response[11], response[12], response[13])
|
||||
} else {
|
||||
glog.Infof("✅ FETCH CORR=%d: Topics count verified OK: %d at response[10:14]=%02x %02x %02x %02x",
|
||||
glog.V(4).Infof("FETCH CORR=%d: Topics count verified OK: %d at response[10:14]=%02x %02x %02x %02x",
|
||||
correlationID, topicsCount, response[10], response[11], response[12], response[13])
|
||||
}
|
||||
}
|
||||
|
||||
Debug("Fetch v%d response constructed, size: %d bytes (flexible: %v)", apiVersion, len(response), isFlexible)
|
||||
glog.Infof("FETCH RESPONSE SUMMARY: correlationID=%d topics=%d totalRecordBytes=%d totalResponseBytes=%d", correlationID, topicsCount, totalAppendedRecordBytes, len(response))
|
||||
glog.V(4).Infof("FETCH RESPONSE SUMMARY: correlationID=%d topics=%d totalRecordBytes=%d totalResponseBytes=%d", correlationID, topicsCount, totalAppendedRecordBytes, len(response))
|
||||
|
||||
// CRITICAL BYTE ANALYSIS for offset 32 error
|
||||
if len(response) > 36 {
|
||||
fmt.Printf("\n🔍🔍🔍 CRITICAL: Analyzing bytes around offset 32:\n")
|
||||
fmt.Printf(" Offset 28-31 (should be after first topic name): %02x %02x %02x %02x\n",
|
||||
response[28], response[29], response[30], response[31])
|
||||
fmt.Printf(" Offset 32-35 (ERROR LOCATION): %02x %02x %02x %02x\n",
|
||||
response[32], response[33], response[34], response[35])
|
||||
fmt.Printf(" Offset 36-39 (should be partition ID or next field): %02x %02x %02x %02x\n",
|
||||
response[36], response[37], response[38], response[39])
|
||||
|
||||
// Decode as int32 (what partition count/ID would be)
|
||||
val32at32 := binary.BigEndian.Uint32(response[32:36])
|
||||
fmt.Printf(" Value at offset 32 as int32: %d (0x%08x)\n", val32at32, val32at32)
|
||||
|
||||
// Decode as int16 (what a string length would be)
|
||||
val16at32 := binary.BigEndian.Uint16(response[32:34])
|
||||
fmt.Printf(" Value at offset 32 as int16: %d (0x%04x)\n", val16at32, val16at32)
|
||||
}
|
||||
|
||||
// HEX DUMP for debugging decode errors: dump first 100 bytes of response to diagnose "invalid length (off=32, len=36)" error
|
||||
if len(response) > 0 && len(response) <= 200 {
|
||||
// Full dump for small responses (likely empty/error responses)
|
||||
fmt.Printf("\n🔍 FETCH RESPONSE HEX DUMP (FULL - %d bytes):\n", len(response))
|
||||
for i := 0; i < len(response); i += 16 {
|
||||
end := i + 16
|
||||
if end > len(response) {
|
||||
end = len(response)
|
||||
}
|
||||
fmt.Printf(" %04d: %02x\n", i, response[i:end])
|
||||
}
|
||||
|
||||
// Decode structure for Fetch v7 (non-flexible)
|
||||
if !isFlexible && len(response) >= 40 {
|
||||
fmt.Printf(" Decoded structure:\n")
|
||||
fmt.Printf(" [0-3] Throttle time ms: %d\n", int32(binary.BigEndian.Uint32(response[0:4])))
|
||||
fmt.Printf(" [4-7] Num topics: %d\n", int32(binary.BigEndian.Uint32(response[4:8])))
|
||||
|
||||
if len(response) >= 12 {
|
||||
topicNameLen := int32(binary.BigEndian.Uint16(response[8:10]))
|
||||
fmt.Printf(" [8-9] Topic name len: %d\n", topicNameLen)
|
||||
if topicNameLen > 0 && 10+topicNameLen <= int32(len(response)) {
|
||||
fmt.Printf(" [10-%d] Topic name: %s\n", 10+topicNameLen-1, string(response[10:10+topicNameLen]))
|
||||
|
||||
partitionsOffset := 10 + topicNameLen
|
||||
if int(partitionsOffset)+4 <= len(response) {
|
||||
numPartitions := int32(binary.BigEndian.Uint32(response[partitionsOffset : partitionsOffset+4]))
|
||||
fmt.Printf(" [%d-%d] Num partitions: %d\n", partitionsOffset, partitionsOffset+3, numPartitions)
|
||||
|
||||
// First partition data starts here
|
||||
partDataOffset := partitionsOffset + 4
|
||||
if int(partDataOffset)+32 <= len(response) {
|
||||
fmt.Printf(" [%d] *** BYTE 32 area (ERROR LOCATION) ***\n", 32)
|
||||
if 32 < len(response) {
|
||||
fmt.Printf(" Byte[32] = 0x%02x (%d)\n", response[32], response[32])
|
||||
}
|
||||
if 33 < len(response) {
|
||||
fmt.Printf(" Byte[33] = 0x%02x (%d)\n", response[33], response[33])
|
||||
}
|
||||
if 34 < len(response) {
|
||||
fmt.Printf(" Byte[34] = 0x%02x (%d)\n", response[34], response[34])
|
||||
}
|
||||
if 35 < len(response) {
|
||||
fmt.Printf(" Byte[35] = 0x%02x (%d)\n", response[35], response[35])
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else if len(response) > 200 {
|
||||
// Partial dump for large responses
|
||||
fmt.Printf("\n🔍 FETCH RESPONSE HEX DUMP (FIRST 64 bytes of %d total):\n", len(response))
|
||||
dumpSize := 64
|
||||
if len(response) < dumpSize {
|
||||
dumpSize = len(response)
|
||||
}
|
||||
for i := 0; i < dumpSize; i += 16 {
|
||||
end := i + 16
|
||||
if end > dumpSize {
|
||||
end = dumpSize
|
||||
}
|
||||
fmt.Printf(" %04d: %02x\n", i, response[i:end])
|
||||
}
|
||||
|
||||
// Always show byte 32 area since that's where the error occurs
|
||||
// Debug byte analysis (disabled for production to reduce CPU usage)
|
||||
// Uncomment for debugging protocol issues
|
||||
/*
|
||||
if len(response) > 36 {
|
||||
fmt.Printf(" *** BYTE 32 area (ERROR LOCATION) ***\n")
|
||||
fmt.Printf(" 0032: %02x\n", response[32:36])
|
||||
glog.V(4).Infof("FETCH CORR=%d: Offset 32-35: %02x %02x %02x %02x",
|
||||
correlationID, response[32], response[33], response[34], response[35])
|
||||
}
|
||||
}
|
||||
*/
|
||||
|
||||
glog.Infof("✅ FETCH RESPONSE COMPLETE: correlationID=%d version=%d size=%d bytes", correlationID, apiVersion, len(response))
|
||||
// HEX DUMP disabled for production (causes high CPU)
|
||||
// Uncomment for debugging protocol issues
|
||||
/*
|
||||
if len(response) > 0 {
|
||||
// Debug hex dump code here...
|
||||
}
|
||||
*/
|
||||
|
||||
glog.V(4).Infof("FETCH RESPONSE COMPLETE: correlationID=%d version=%d size=%d bytes", correlationID, apiVersion, len(response))
|
||||
return response, nil
|
||||
}
|
||||
|
||||
|
@@ -394,7 +394,6 @@ func (h *Handler) GetConnectionContext() *integration.ConnectionContext {
|
||||
// HandleConn processes a single client connection
|
||||
func (h *Handler) HandleConn(ctx context.Context, conn net.Conn) error {
|
||||
connectionID := fmt.Sprintf("%s->%s", conn.RemoteAddr(), conn.LocalAddr())
|
||||
fmt.Printf("🔥🔥🔥 HandleConn START: %s\n", connectionID)
|
||||
Debug("KAFKA 8.0.0 DEBUG: NEW HANDLER CODE ACTIVE - %s", time.Now().Format("15:04:05"))
|
||||
|
||||
// Record connection metrics
|
||||
@@ -409,35 +408,28 @@ func (h *Handler) HandleConn(ctx context.Context, conn net.Conn) error {
|
||||
|
||||
// CRITICAL: Create per-connection BrokerClient for isolated gRPC streams
|
||||
// This prevents different connections from interfering with each other's Fetch requests
|
||||
fmt.Printf("🔥🔥🔥 [%s] [BROKER_CLIENT] Creating per-connection BrokerClient\n", connectionID)
|
||||
glog.Infof("[%s] [BROKER_CLIENT] Creating per-connection BrokerClient", connectionID)
|
||||
glog.V(4).Infof("[%s] [BROKER_CLIENT] Creating per-connection BrokerClient", connectionID)
|
||||
connBrokerClient, err := h.seaweedMQHandler.CreatePerConnectionBrokerClient()
|
||||
if err != nil {
|
||||
fmt.Printf("🔥🔥🔥 [%s] [BROKER_CLIENT] Failed: %v\n", connectionID, err)
|
||||
glog.Errorf("[%s] [BROKER_CLIENT] Failed to create per-connection BrokerClient: %v", connectionID, err)
|
||||
return fmt.Errorf("failed to create broker client: %w", err)
|
||||
}
|
||||
h.connContext.BrokerClient = connBrokerClient
|
||||
fmt.Printf("🔥🔥🔥 [%s] [BROKER_CLIENT] Per-connection BrokerClient created successfully\n", connectionID)
|
||||
glog.Infof("[%s] [BROKER_CLIENT] Per-connection BrokerClient created successfully", connectionID)
|
||||
glog.V(4).Infof("[%s] [BROKER_CLIENT] Per-connection BrokerClient created successfully", connectionID)
|
||||
|
||||
Debug("[%s] NEW CONNECTION ESTABLISHED", connectionID)
|
||||
|
||||
defer func() {
|
||||
glog.V(0).Infof("🔍 [%s] Connection closing, cleaning up BrokerClient", connectionID)
|
||||
Debug("[%s] Connection closing, cleaning up BrokerClient", connectionID)
|
||||
// Close the per-connection broker client
|
||||
if connBrokerClient != nil {
|
||||
glog.V(0).Infof("🔍 [%s] Calling BrokerClient.Close()", connectionID)
|
||||
if closeErr := connBrokerClient.Close(); closeErr != nil {
|
||||
glog.Errorf("🔍 [%s] Error closing BrokerClient: %v", connectionID, closeErr)
|
||||
Error("[%s] Error closing BrokerClient: %v", connectionID, closeErr)
|
||||
}
|
||||
}
|
||||
RecordDisconnectionMetrics()
|
||||
h.connContext = nil // Clear connection context
|
||||
conn.Close()
|
||||
glog.V(0).Infof("🔍 [%s] Connection cleanup complete", connectionID)
|
||||
}()
|
||||
|
||||
r := bufio.NewReader(conn)
|
||||
@@ -549,8 +541,8 @@ func (h *Handler) HandleConn(ctx context.Context, conn net.Conn) error {
|
||||
defer func() {
|
||||
close(controlChan)
|
||||
close(dataChan)
|
||||
close(responseChan) // Close BEFORE wg.Wait() so response writer can exit
|
||||
wg.Wait()
|
||||
close(responseChan)
|
||||
}()
|
||||
|
||||
for {
|
||||
@@ -622,7 +614,7 @@ func (h *Handler) HandleConn(ctx context.Context, conn net.Conn) error {
|
||||
// Successfully read the message size
|
||||
size := binary.BigEndian.Uint32(sizeBytes[:])
|
||||
Debug("[%s] Read message size header: %d bytes", connectionID, size)
|
||||
fmt.Printf("🔥 PROTOCOL DEBUG: Read message size: %d bytes\n", size)
|
||||
// Debug("Read message size: %d bytes", size)
|
||||
if size == 0 || size > 1024*1024 { // 1MB limit
|
||||
// Use standardized error for message size limit
|
||||
Debug("[%s] Invalid message size: %d (limit: 1MB)", connectionID, size)
|
||||
@@ -657,8 +649,7 @@ func (h *Handler) HandleConn(ctx context.Context, conn net.Conn) error {
|
||||
apiVersion := binary.BigEndian.Uint16(messageBuf[2:4])
|
||||
correlationID := binary.BigEndian.Uint32(messageBuf[4:8])
|
||||
|
||||
apiName := getAPIName(apiKey)
|
||||
fmt.Printf("🔥 PROTOCOL DEBUG: Parsed header - API Key: %d (%s), Version: %d, Correlation: %d\n", apiKey, apiName, apiVersion, correlationID)
|
||||
// Debug("Parsed header - API Key: %d (%s), Version: %d, Correlation: %d", apiKey, getAPIName(apiKey), apiVersion, correlationID)
|
||||
|
||||
// Validate API version against what we support
|
||||
Debug("VALIDATING API VERSION: Key=%d, Version=%d", apiKey, apiVersion)
|
||||
@@ -3057,14 +3048,17 @@ func (h *Handler) writeResponseWithHeader(w *bufio.Writer, correlationID uint32,
|
||||
// Write response body
|
||||
fullResponse = append(fullResponse, responseBody...)
|
||||
|
||||
// Hex dump for debugging (first 64 bytes)
|
||||
dumpLen := len(fullResponse)
|
||||
if dumpLen > 64 {
|
||||
dumpLen = 64
|
||||
}
|
||||
if apiKey == 1 || apiKey == 19 { // Fetch or CreateTopics
|
||||
glog.Infof("🔍 API %d v%d response wire format (first %d bytes):\n%s", apiKey, apiVersion, dumpLen, hexDump(fullResponse[:dumpLen]))
|
||||
}
|
||||
// Hex dump for debugging (disabled for production to reduce CPU)
|
||||
// Uncomment for debugging protocol issues
|
||||
/*
|
||||
dumpLen := len(fullResponse)
|
||||
if dumpLen > 64 {
|
||||
dumpLen = 64
|
||||
}
|
||||
if apiKey == 1 || apiKey == 19 { // Fetch or CreateTopics
|
||||
glog.V(4).Infof("API %d v%d response wire format (first %d bytes):\n%s", apiKey, apiVersion, dumpLen, hexDump(fullResponse[:dumpLen]))
|
||||
}
|
||||
*/
|
||||
Debug("Wrote API %d response v%d: size=%d, flexible=%t, correlationID=%d, totalBytes=%d", apiKey, apiVersion, totalSize, isFlexible, correlationID, len(fullResponse))
|
||||
|
||||
// Write to connection
|
||||
|
@@ -2,6 +2,7 @@ package topic
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
)
|
||||
|
||||
@@ -22,15 +23,15 @@ func (localTopic *LocalTopic) findPartition(partition Partition) *LocalPartition
|
||||
localTopic.partitionLock.RLock()
|
||||
defer localTopic.partitionLock.RUnlock()
|
||||
|
||||
glog.V(0).Infof("🔍 DEBUG: findPartition searching for %s in %d partitions", partition.String(), len(localTopic.Partitions))
|
||||
glog.V(4).Infof("findPartition searching for %s in %d partitions", partition.String(), len(localTopic.Partitions))
|
||||
for i, localPartition := range localTopic.Partitions {
|
||||
glog.V(0).Infof("🔍 DEBUG: Comparing partition[%d]: %s with target %s", i, localPartition.Partition.String(), partition.String())
|
||||
glog.V(4).Infof("Comparing partition[%d]: %s with target %s", i, localPartition.Partition.String(), partition.String())
|
||||
if localPartition.Partition.LogicalEquals(partition) {
|
||||
glog.V(0).Infof("🔍 DEBUG: Found matching partition at index %d", i)
|
||||
glog.V(4).Infof("Found matching partition at index %d", i)
|
||||
return localPartition
|
||||
}
|
||||
}
|
||||
glog.V(0).Infof("🔍 DEBUG: No matching partition found for %s", partition.String())
|
||||
glog.V(4).Infof("No matching partition found for %s", partition.String())
|
||||
return nil
|
||||
}
|
||||
func (localTopic *LocalTopic) removePartition(partition Partition) bool {
|
||||
|
@@ -16,7 +16,7 @@ import (
|
||||
// DistributedLock is a grpc handler to handle FilerServer's LockRequest
|
||||
func (fs *FilerServer) DistributedLock(ctx context.Context, req *filer_pb.LockRequest) (resp *filer_pb.LockResponse, err error) {
|
||||
|
||||
glog.V(0).Infof("🔥 FILER LOCK: Received DistributedLock request - name=%s owner=%s renewToken=%s secondsToLock=%d isMoved=%v",
|
||||
glog.V(4).Infof("FILER LOCK: Received DistributedLock request - name=%s owner=%s renewToken=%s secondsToLock=%d isMoved=%v",
|
||||
req.Name, req.Owner, req.RenewToken, req.SecondsToLock, req.IsMoved)
|
||||
|
||||
resp = &filer_pb.LockResponse{}
|
||||
@@ -24,11 +24,11 @@ func (fs *FilerServer) DistributedLock(ctx context.Context, req *filer_pb.LockRe
|
||||
var movedTo pb.ServerAddress
|
||||
expiredAtNs := time.Now().Add(time.Duration(req.SecondsToLock) * time.Second).UnixNano()
|
||||
resp.LockOwner, resp.RenewToken, movedTo, err = fs.filer.Dlm.LockWithTimeout(req.Name, expiredAtNs, req.RenewToken, req.Owner)
|
||||
glog.V(0).Infof("🔍 FILER LOCK: LockWithTimeout result - name=%s lockOwner=%s renewToken=%s movedTo=%s err=%v",
|
||||
glog.V(4).Infof("FILER LOCK: LockWithTimeout result - name=%s lockOwner=%s renewToken=%s movedTo=%s err=%v",
|
||||
req.Name, resp.LockOwner, resp.RenewToken, movedTo, err)
|
||||
glog.V(4).Infof("lock %s %v %v %v, isMoved=%v %v", req.Name, req.SecondsToLock, req.RenewToken, req.Owner, req.IsMoved, movedTo)
|
||||
if movedTo != "" && movedTo != fs.option.Host && !req.IsMoved {
|
||||
glog.V(0).Infof("🔄 FILER LOCK: Forwarding to correct filer - from=%s to=%s", fs.option.Host, movedTo)
|
||||
glog.V(0).Infof("FILER LOCK: Forwarding to correct filer - from=%s to=%s", fs.option.Host, movedTo)
|
||||
err = pb.WithFilerClient(false, 0, movedTo, fs.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
|
||||
secondResp, err := client.DistributedLock(context.Background(), &filer_pb.LockRequest{
|
||||
Name: req.Name,
|
||||
@@ -41,9 +41,9 @@ func (fs *FilerServer) DistributedLock(ctx context.Context, req *filer_pb.LockRe
|
||||
resp.RenewToken = secondResp.RenewToken
|
||||
resp.LockOwner = secondResp.LockOwner
|
||||
resp.Error = secondResp.Error
|
||||
glog.V(0).Infof("✅ FILER LOCK: Forwarded lock acquired - name=%s renewToken=%s", req.Name, resp.RenewToken)
|
||||
glog.V(0).Infof("FILER LOCK: Forwarded lock acquired - name=%s renewToken=%s", req.Name, resp.RenewToken)
|
||||
} else {
|
||||
glog.V(0).Infof("❌ FILER LOCK: Forward failed - name=%s err=%v", req.Name, err)
|
||||
glog.V(0).Infof("FILER LOCK: Forward failed - name=%s err=%v", req.Name, err)
|
||||
}
|
||||
return err
|
||||
})
|
||||
@@ -51,13 +51,13 @@ func (fs *FilerServer) DistributedLock(ctx context.Context, req *filer_pb.LockRe
|
||||
|
||||
if err != nil {
|
||||
resp.Error = fmt.Sprintf("%v", err)
|
||||
glog.V(0).Infof("❌ FILER LOCK: Error - name=%s error=%s", req.Name, resp.Error)
|
||||
glog.V(0).Infof("FILER LOCK: Error - name=%s error=%s", req.Name, resp.Error)
|
||||
}
|
||||
if movedTo != "" {
|
||||
resp.LockHostMovedTo = string(movedTo)
|
||||
}
|
||||
|
||||
glog.V(0).Infof("📤 FILER LOCK: Returning response - name=%s renewToken=%s lockOwner=%s error=%s movedTo=%s",
|
||||
glog.V(4).Infof("FILER LOCK: Returning response - name=%s renewToken=%s lockOwner=%s error=%s movedTo=%s",
|
||||
req.Name, resp.RenewToken, resp.LockOwner, resp.Error, resp.LockHostMovedTo)
|
||||
|
||||
return resp, nil
|
||||
|
Reference in New Issue
Block a user