Browse Source

Merge branch 'master' into simplify-worker

pull/7394/head
Mariano Ntrougkas 1 month ago
committed by GitHub
parent
commit
293e9a8168
No known key found for this signature in database GPG Key ID: B5690EEEBB952194
  1. 2
      k8s/charts/seaweedfs/values.yaml
  2. 13
      weed/admin/dash/ec_shard_management.go
  3. 1
      weed/admin/handlers/maintenance_handlers.go
  4. 4
      weed/admin/maintenance/maintenance_integration.go
  5. 5
      weed/cluster/master_client.go
  6. 28
      weed/command/autocomplete.go
  7. 1
      weed/mq/broker/broker_grpc_sub.go
  8. 12
      weed/mq/kafka/consumer/incremental_rebalancing_test.go
  9. 90
      weed/mq/kafka/consumer/rebalance_timeout.go
  10. 112
      weed/mq/kafka/consumer/rebalance_timeout_test.go
  11. 1
      weed/mq/kafka/consumer_offset/memory_storage.go
  12. 1
      weed/mq/kafka/consumer_offset/memory_storage_test.go
  13. 1
      weed/mq/kafka/consumer_offset/storage.go
  14. 1
      weed/mq/kafka/gateway/test_mock_handler.go
  15. 2
      weed/mq/kafka/package.go
  16. 2
      weed/mq/kafka/partition_mapping.go
  17. 2
      weed/mq/kafka/protocol/describe_cluster.go
  18. 1
      weed/mq/kafka/protocol/flexible_versions.go
  19. 18
      weed/mq/kafka/protocol/group_introspection.go
  20. 35
      weed/mq/kafka/protocol/handler.go
  21. 1
      weed/mq/kafka/protocol/offset_storage_adapter.go
  22. 1
      weed/mq/kafka/protocol/response_validation_example_test.go
  23. 72
      weed/mq/kafka/schema/envelope_test.go
  24. 2
      weed/mq/kafka/schema/envelope_varint_test.go
  25. 2
      weed/mq/metadata_constants.go
  26. 54
      weed/mq/offset/migration.go
  27. 10
      weed/mq/schema/flat_schema_utils_test.go
  28. 2
      weed/pb/mq_agent_pb/publish_response_test.go
  29. 6
      weed/pb/schema_pb/offset_test.go
  30. 7
      weed/pb/volume_server.proto
  31. 502
      weed/pb/volume_server_pb/volume_server.pb.go
  32. 18
      weed/remote_storage/azure/azure_storage_client_test.go
  33. 2
      weed/s3api/s3_sse_s3_integration_test.go
  34. 10
      weed/s3api/s3_validation_utils.go
  35. 64
      weed/server/volume_grpc_erasure_coding.go
  36. 14
      weed/shell/command_ec_common.go
  37. 3
      weed/shell/command_ec_rebuild.go
  38. 13
      weed/storage/disk_location_ec.go
  39. 46
      weed/storage/erasure_coding/ec_context.go
  40. 113
      weed/storage/erasure_coding/ec_encoder.go
  41. 19
      weed/storage/erasure_coding/ec_test.go
  42. 28
      weed/storage/erasure_coding/ec_volume.go
  43. 21
      weed/storage/erasure_coding/ec_volume_info.go
  44. 3
      weed/storage/store_ec.go
  45. 11
      weed/topology/topology_ec.go
  46. 202
      weed/util/log_buffer/log_buffer_flush_gap_test.go
  47. 2
      weed/util/version/constants.go
  48. 1
      weed/worker/client.go
  49. 3
      weed/worker/tasks/erasure_coding/ec_task.go

2
k8s/charts/seaweedfs/values.yaml

@ -873,7 +873,7 @@ filer:
# anonymousRead: false # anonymousRead: false
s3: s3:
enabled: true
enabled: false
imageOverride: null imageOverride: null
restartPolicy: null restartPolicy: null
replicas: 1 replicas: 1

13
weed/admin/dash/ec_shard_management.go

@ -68,7 +68,7 @@ func (s *AdminServer) GetClusterEcShards(page int, pageSize int, sortBy string,
// Create individual shard entries for each shard this server has // Create individual shard entries for each shard this server has
shardBits := ecShardInfo.EcIndexBits shardBits := ecShardInfo.EcIndexBits
for shardId := 0; shardId < erasure_coding.TotalShardsCount; shardId++ {
for shardId := 0; shardId < erasure_coding.MaxShardCount; shardId++ {
if (shardBits & (1 << uint(shardId))) != 0 { if (shardBits & (1 << uint(shardId))) != 0 {
// Mark this shard as present for this volume // Mark this shard as present for this volume
volumeShardsMap[volumeId][shardId] = true volumeShardsMap[volumeId][shardId] = true
@ -112,6 +112,7 @@ func (s *AdminServer) GetClusterEcShards(page int, pageSize int, sortBy string,
shardCount := len(shardsPresent) shardCount := len(shardsPresent)
// Find which shards are missing for this volume across ALL servers // Find which shards are missing for this volume across ALL servers
// Uses default 10+4 (14 total shards)
for shardId := 0; shardId < erasure_coding.TotalShardsCount; shardId++ { for shardId := 0; shardId < erasure_coding.TotalShardsCount; shardId++ {
if !shardsPresent[shardId] { if !shardsPresent[shardId] {
missingShards = append(missingShards, shardId) missingShards = append(missingShards, shardId)
@ -332,7 +333,7 @@ func (s *AdminServer) GetClusterEcVolumes(page int, pageSize int, sortBy string,
// Process each shard this server has for this volume // Process each shard this server has for this volume
shardBits := ecShardInfo.EcIndexBits shardBits := ecShardInfo.EcIndexBits
for shardId := 0; shardId < erasure_coding.TotalShardsCount; shardId++ {
for shardId := 0; shardId < erasure_coding.MaxShardCount; shardId++ {
if (shardBits & (1 << uint(shardId))) != 0 { if (shardBits & (1 << uint(shardId))) != 0 {
// Record shard location // Record shard location
volume.ShardLocations[shardId] = node.Id volume.ShardLocations[shardId] = node.Id
@ -392,7 +393,7 @@ func (s *AdminServer) GetClusterEcVolumes(page int, pageSize int, sortBy string,
for _, volume := range volumeData { for _, volume := range volumeData {
volume.TotalShards = len(volume.ShardLocations) volume.TotalShards = len(volume.ShardLocations)
// Find missing shards
// Find missing shards (default 10+4 = 14 total shards)
var missingShards []int var missingShards []int
for shardId := 0; shardId < erasure_coding.TotalShardsCount; shardId++ { for shardId := 0; shardId < erasure_coding.TotalShardsCount; shardId++ {
if _, exists := volume.ShardLocations[shardId]; !exists { if _, exists := volume.ShardLocations[shardId]; !exists {
@ -523,7 +524,7 @@ func sortEcVolumes(volumes []EcVolumeWithShards, sortBy string, sortOrder string
// getShardCount returns the number of shards represented by the bitmap // getShardCount returns the number of shards represented by the bitmap
func getShardCount(ecIndexBits uint32) int { func getShardCount(ecIndexBits uint32) int {
count := 0 count := 0
for i := 0; i < erasure_coding.TotalShardsCount; i++ {
for i := 0; i < erasure_coding.MaxShardCount; i++ {
if (ecIndexBits & (1 << uint(i))) != 0 { if (ecIndexBits & (1 << uint(i))) != 0 {
count++ count++
} }
@ -532,6 +533,7 @@ func getShardCount(ecIndexBits uint32) int {
} }
// getMissingShards returns a slice of missing shard IDs for a volume // getMissingShards returns a slice of missing shard IDs for a volume
// Assumes default 10+4 EC configuration (14 total shards)
func getMissingShards(ecIndexBits uint32) []int { func getMissingShards(ecIndexBits uint32) []int {
var missing []int var missing []int
for i := 0; i < erasure_coding.TotalShardsCount; i++ { for i := 0; i < erasure_coding.TotalShardsCount; i++ {
@ -614,7 +616,7 @@ func (s *AdminServer) GetEcVolumeDetails(volumeID uint32, sortBy string, sortOrd
// Create individual shard entries for each shard this server has // Create individual shard entries for each shard this server has
shardBits := ecShardInfo.EcIndexBits shardBits := ecShardInfo.EcIndexBits
for shardId := 0; shardId < erasure_coding.TotalShardsCount; shardId++ {
for shardId := 0; shardId < erasure_coding.MaxShardCount; shardId++ {
if (shardBits & (1 << uint(shardId))) != 0 { if (shardBits & (1 << uint(shardId))) != 0 {
ecShard := EcShardWithInfo{ ecShard := EcShardWithInfo{
VolumeID: ecShardInfo.Id, VolumeID: ecShardInfo.Id,
@ -698,6 +700,7 @@ func (s *AdminServer) GetEcVolumeDetails(volumeID uint32, sortBy string, sortOrd
} }
totalUniqueShards := len(foundShards) totalUniqueShards := len(foundShards)
// Check completeness using default 10+4 (14 total shards)
isComplete := (totalUniqueShards == erasure_coding.TotalShardsCount) isComplete := (totalUniqueShards == erasure_coding.TotalShardsCount)
// Calculate missing shards // Calculate missing shards

1
weed/admin/handlers/maintenance_handlers.go

@ -46,7 +46,6 @@ func (h *MaintenanceHandlers) ShowTaskDetail(c *gin.Context) {
return return
} }
c.Header("Content-Type", "text/html") c.Header("Content-Type", "text/html")
taskDetailComponent := app.TaskDetail(taskDetail) taskDetailComponent := app.TaskDetail(taskDetail)
layoutComponent := layout.Layout(c, taskDetailComponent) layoutComponent := layout.Layout(c, taskDetailComponent)

4
weed/admin/maintenance/maintenance_integration.go

@ -306,25 +306,21 @@ func (s *MaintenanceIntegration) CanScheduleWithTaskSchedulers(task *Maintenance
return false // Fallback to existing logic for unknown types return false // Fallback to existing logic for unknown types
} }
// Convert task objects // Convert task objects
taskObject := s.convertTaskToTaskSystem(task) taskObject := s.convertTaskToTaskSystem(task)
if taskObject == nil { if taskObject == nil {
return false return false
} }
runningTaskObjects := s.convertTasksToTaskSystem(runningTasks) runningTaskObjects := s.convertTasksToTaskSystem(runningTasks)
workerObjects := s.convertWorkersToTaskSystem(availableWorkers) workerObjects := s.convertWorkersToTaskSystem(availableWorkers)
// Get the appropriate scheduler // Get the appropriate scheduler
scheduler := s.taskRegistry.GetScheduler(taskType) scheduler := s.taskRegistry.GetScheduler(taskType)
if scheduler == nil { if scheduler == nil {
return false return false
} }
canSchedule := scheduler.CanScheduleNow(taskObject, runningTaskObjects, workerObjects) canSchedule := scheduler.CanScheduleNow(taskObject, runningTaskObjects, workerObjects)
return canSchedule return canSchedule

5
weed/cluster/master_client.go

@ -16,6 +16,9 @@ func ListExistingPeerUpdates(master pb.ServerAddress, grpcDialOption grpc.DialOp
ClientType: clientType, ClientType: clientType,
FilerGroup: filerGroup, FilerGroup: filerGroup,
}) })
if err != nil {
return err
}
glog.V(0).Infof("the cluster has %d %s\n", len(resp.ClusterNodes), clientType) glog.V(0).Infof("the cluster has %d %s\n", len(resp.ClusterNodes), clientType)
for _, node := range resp.ClusterNodes { for _, node := range resp.ClusterNodes {
@ -26,7 +29,7 @@ func ListExistingPeerUpdates(master pb.ServerAddress, grpcDialOption grpc.DialOp
CreatedAtNs: node.CreatedAtNs, CreatedAtNs: node.CreatedAtNs,
}) })
} }
return err
return nil
}); grpcErr != nil { }); grpcErr != nil {
glog.V(0).Infof("connect to %s: %v", master, grpcErr) glog.V(0).Infof("connect to %s: %v", master, grpcErr)
} }

28
weed/command/autocomplete.go

@ -2,11 +2,11 @@ package command
import ( import (
"fmt" "fmt"
"os"
"path/filepath"
"github.com/posener/complete" "github.com/posener/complete"
completeinstall "github.com/posener/complete/cmd/install" completeinstall "github.com/posener/complete/cmd/install"
flag "github.com/seaweedfs/seaweedfs/weed/util/fla9" flag "github.com/seaweedfs/seaweedfs/weed/util/fla9"
"os"
"path/filepath"
"runtime" "runtime"
) )
@ -53,14 +53,14 @@ func printAutocompleteScript(shell string) bool {
return false return false
} }
switch shell {
case "bash":
fmt.Printf("complete -C %q weed\n", binPath)
case "zsh":
fmt.Printf("autoload -U +X bashcompinit && bashcompinit\n")
fmt.Printf("complete -o nospace -C %q weed\n", binPath)
case "fish":
fmt.Printf(`function __complete_weed
switch shell {
case "bash":
fmt.Printf("complete -C %q weed\n", binPath)
case "zsh":
fmt.Printf("autoload -U +X bashcompinit && bashcompinit\n")
fmt.Printf("complete -o nospace -C %q weed\n", binPath)
case "fish":
fmt.Printf(`function __complete_weed
set -lx COMP_LINE (commandline -cp) set -lx COMP_LINE (commandline -cp)
test -z (commandline -ct) test -z (commandline -ct)
and set COMP_LINE "$COMP_LINE " and set COMP_LINE "$COMP_LINE "
@ -68,10 +68,10 @@ func printAutocompleteScript(shell string) bool {
end end
complete -f -c weed -a "(__complete_weed)" complete -f -c weed -a "(__complete_weed)"
`, binPath) `, binPath)
default:
fmt.Fprintf(os.Stderr, "unsupported shell: %s. Supported shells: bash, zsh, fish\n", shell)
return false
}
default:
fmt.Fprintf(os.Stderr, "unsupported shell: %s. Supported shells: bash, zsh, fish\n", shell)
return false
}
return true return true
} }

1
weed/mq/broker/broker_grpc_sub.go

@ -272,7 +272,6 @@ subscribeLoop:
TsNs: logEntry.TsNs, TsNs: logEntry.TsNs,
} }
if err := stream.Send(&mq_pb.SubscribeMessageResponse{Message: &mq_pb.SubscribeMessageResponse_Data{ if err := stream.Send(&mq_pb.SubscribeMessageResponse{Message: &mq_pb.SubscribeMessageResponse_Data{
Data: dataMsg, Data: dataMsg,
}}); err != nil { }}); err != nil {

12
weed/mq/kafka/consumer/incremental_rebalancing_test.go

@ -103,15 +103,15 @@ func TestIncrementalCooperativeAssignmentStrategy_RebalanceWithRevocation(t *tes
t.Errorf("Expected member-2 to have 0 partitions during revocation, got %d", len(member2Assignments)) t.Errorf("Expected member-2 to have 0 partitions during revocation, got %d", len(member2Assignments))
} }
t.Logf("Revocation phase - Member-1: %d partitions, Member-2: %d partitions",
t.Logf("Revocation phase - Member-1: %d partitions, Member-2: %d partitions",
len(member1Assignments), len(member2Assignments)) len(member1Assignments), len(member2Assignments))
// Simulate time passing and second call (should move to assignment phase) // Simulate time passing and second call (should move to assignment phase)
time.Sleep(10 * time.Millisecond) time.Sleep(10 * time.Millisecond)
// Force move to assignment phase by setting timeout to 0 // Force move to assignment phase by setting timeout to 0
state.RevocationTimeout = 0 state.RevocationTimeout = 0
assignments2 := strategy.Assign(members, topicPartitions) assignments2 := strategy.Assign(members, topicPartitions)
// Should complete rebalance // Should complete rebalance
@ -136,7 +136,7 @@ func TestIncrementalCooperativeAssignmentStrategy_RebalanceWithRevocation(t *tes
t.Errorf("Expected 4 total partitions after rebalance, got %d", totalFinalPartitions) t.Errorf("Expected 4 total partitions after rebalance, got %d", totalFinalPartitions)
} }
t.Logf("Final assignment - Member-1: %d partitions, Member-2: %d partitions",
t.Logf("Final assignment - Member-1: %d partitions, Member-2: %d partitions",
len(member1FinalAssignments), len(member2FinalAssignments)) len(member1FinalAssignments), len(member2FinalAssignments))
} }
@ -239,7 +239,7 @@ func TestIncrementalCooperativeAssignmentStrategy_MultipleTopics(t *testing.T) {
t.Errorf("Expected partition %s to be assigned", expected) t.Errorf("Expected partition %s to be assigned", expected)
} }
} }
// Debug: Print all assigned partitions // Debug: Print all assigned partitions
t.Logf("All assigned partitions: %v", allAssignedPartitions) t.Logf("All assigned partitions: %v", allAssignedPartitions)
} }
@ -390,7 +390,7 @@ func TestIncrementalCooperativeAssignmentStrategy_StateTransitions(t *testing.T)
// Force timeout to move to assignment phase // Force timeout to move to assignment phase
state.RevocationTimeout = 0 state.RevocationTimeout = 0
strategy.Assign(members, topicPartitions) strategy.Assign(members, topicPartitions)
// Should complete and return to None // Should complete and return to None
state = strategy.GetRebalanceState() state = strategy.GetRebalanceState()
if state.Phase != RebalancePhaseNone { if state.Phase != RebalancePhaseNone {

90
weed/mq/kafka/consumer/rebalance_timeout.go

@ -24,12 +24,12 @@ func (rtm *RebalanceTimeoutManager) CheckRebalanceTimeouts() {
for _, group := range rtm.coordinator.groups { for _, group := range rtm.coordinator.groups {
group.Mu.Lock() group.Mu.Lock()
// Only check timeouts for groups in rebalancing states // Only check timeouts for groups in rebalancing states
if group.State == GroupStatePreparingRebalance || group.State == GroupStateCompletingRebalance { if group.State == GroupStatePreparingRebalance || group.State == GroupStateCompletingRebalance {
rtm.checkGroupRebalanceTimeout(group, now) rtm.checkGroupRebalanceTimeout(group, now)
} }
group.Mu.Unlock() group.Mu.Unlock()
} }
} }
@ -37,7 +37,7 @@ func (rtm *RebalanceTimeoutManager) CheckRebalanceTimeouts() {
// checkGroupRebalanceTimeout checks and handles rebalance timeout for a specific group // checkGroupRebalanceTimeout checks and handles rebalance timeout for a specific group
func (rtm *RebalanceTimeoutManager) checkGroupRebalanceTimeout(group *ConsumerGroup, now time.Time) { func (rtm *RebalanceTimeoutManager) checkGroupRebalanceTimeout(group *ConsumerGroup, now time.Time) {
expiredMembers := make([]string, 0) expiredMembers := make([]string, 0)
for memberID, member := range group.Members { for memberID, member := range group.Members {
// Check if member has exceeded its rebalance timeout // Check if member has exceeded its rebalance timeout
rebalanceTimeout := time.Duration(member.RebalanceTimeout) * time.Millisecond rebalanceTimeout := time.Duration(member.RebalanceTimeout) * time.Millisecond
@ -45,21 +45,21 @@ func (rtm *RebalanceTimeoutManager) checkGroupRebalanceTimeout(group *ConsumerGr
// Use default rebalance timeout if not specified // Use default rebalance timeout if not specified
rebalanceTimeout = time.Duration(rtm.coordinator.rebalanceTimeoutMs) * time.Millisecond rebalanceTimeout = time.Duration(rtm.coordinator.rebalanceTimeoutMs) * time.Millisecond
} }
// For members in pending state during rebalance, check against join time // For members in pending state during rebalance, check against join time
if member.State == MemberStatePending { if member.State == MemberStatePending {
if now.Sub(member.JoinedAt) > rebalanceTimeout { if now.Sub(member.JoinedAt) > rebalanceTimeout {
expiredMembers = append(expiredMembers, memberID) expiredMembers = append(expiredMembers, memberID)
} }
} }
// Also check session timeout as a fallback // Also check session timeout as a fallback
sessionTimeout := time.Duration(member.SessionTimeout) * time.Millisecond sessionTimeout := time.Duration(member.SessionTimeout) * time.Millisecond
if now.Sub(member.LastHeartbeat) > sessionTimeout { if now.Sub(member.LastHeartbeat) > sessionTimeout {
expiredMembers = append(expiredMembers, memberID) expiredMembers = append(expiredMembers, memberID)
} }
} }
// Remove expired members and trigger rebalance if necessary // Remove expired members and trigger rebalance if necessary
if len(expiredMembers) > 0 { if len(expiredMembers) > 0 {
rtm.evictExpiredMembers(group, expiredMembers) rtm.evictExpiredMembers(group, expiredMembers)
@ -70,13 +70,13 @@ func (rtm *RebalanceTimeoutManager) checkGroupRebalanceTimeout(group *ConsumerGr
func (rtm *RebalanceTimeoutManager) evictExpiredMembers(group *ConsumerGroup, expiredMembers []string) { func (rtm *RebalanceTimeoutManager) evictExpiredMembers(group *ConsumerGroup, expiredMembers []string) {
for _, memberID := range expiredMembers { for _, memberID := range expiredMembers {
delete(group.Members, memberID) delete(group.Members, memberID)
// If the leader was evicted, clear leader // If the leader was evicted, clear leader
if group.Leader == memberID { if group.Leader == memberID {
group.Leader = "" group.Leader = ""
} }
} }
// Update group state based on remaining members // Update group state based on remaining members
if len(group.Members) == 0 { if len(group.Members) == 0 {
group.State = GroupStateEmpty group.State = GroupStateEmpty
@ -92,18 +92,18 @@ func (rtm *RebalanceTimeoutManager) evictExpiredMembers(group *ConsumerGroup, ex
break break
} }
} }
// Reset to preparing rebalance to restart the process // Reset to preparing rebalance to restart the process
group.State = GroupStatePreparingRebalance group.State = GroupStatePreparingRebalance
group.Generation++ group.Generation++
// Mark remaining members as pending // Mark remaining members as pending
for _, member := range group.Members { for _, member := range group.Members {
member.State = MemberStatePending member.State = MemberStatePending
} }
} }
} }
group.LastActivity = time.Now() group.LastActivity = time.Now()
} }
@ -112,7 +112,7 @@ func (rtm *RebalanceTimeoutManager) IsRebalanceStuck(group *ConsumerGroup, maxRe
if group.State != GroupStatePreparingRebalance && group.State != GroupStateCompletingRebalance { if group.State != GroupStatePreparingRebalance && group.State != GroupStateCompletingRebalance {
return false return false
} }
return time.Since(group.LastActivity) > maxRebalanceDuration return time.Since(group.LastActivity) > maxRebalanceDuration
} }
@ -120,14 +120,14 @@ func (rtm *RebalanceTimeoutManager) IsRebalanceStuck(group *ConsumerGroup, maxRe
func (rtm *RebalanceTimeoutManager) ForceCompleteRebalance(group *ConsumerGroup) { func (rtm *RebalanceTimeoutManager) ForceCompleteRebalance(group *ConsumerGroup) {
group.Mu.Lock() group.Mu.Lock()
defer group.Mu.Unlock() defer group.Mu.Unlock()
// If stuck in preparing rebalance, move to completing // If stuck in preparing rebalance, move to completing
if group.State == GroupStatePreparingRebalance { if group.State == GroupStatePreparingRebalance {
group.State = GroupStateCompletingRebalance group.State = GroupStateCompletingRebalance
group.LastActivity = time.Now() group.LastActivity = time.Now()
return return
} }
// If stuck in completing rebalance, force to stable // If stuck in completing rebalance, force to stable
if group.State == GroupStateCompletingRebalance { if group.State == GroupStateCompletingRebalance {
group.State = GroupStateStable group.State = GroupStateStable
@ -145,21 +145,21 @@ func (rtm *RebalanceTimeoutManager) GetRebalanceStatus(groupID string) *Rebalanc
if group == nil { if group == nil {
return nil return nil
} }
group.Mu.RLock() group.Mu.RLock()
defer group.Mu.RUnlock() defer group.Mu.RUnlock()
status := &RebalanceStatus{ status := &RebalanceStatus{
GroupID: groupID,
State: group.State,
Generation: group.Generation,
MemberCount: len(group.Members),
Leader: group.Leader,
LastActivity: group.LastActivity,
IsRebalancing: group.State == GroupStatePreparingRebalance || group.State == GroupStateCompletingRebalance,
GroupID: groupID,
State: group.State,
Generation: group.Generation,
MemberCount: len(group.Members),
Leader: group.Leader,
LastActivity: group.LastActivity,
IsRebalancing: group.State == GroupStatePreparingRebalance || group.State == GroupStateCompletingRebalance,
RebalanceDuration: time.Since(group.LastActivity), RebalanceDuration: time.Since(group.LastActivity),
} }
// Calculate member timeout status // Calculate member timeout status
now := time.Now() now := time.Now()
for memberID, member := range group.Members { for memberID, member := range group.Members {
@ -171,48 +171,48 @@ func (rtm *RebalanceTimeoutManager) GetRebalanceStatus(groupID string) *Rebalanc
SessionTimeout: time.Duration(member.SessionTimeout) * time.Millisecond, SessionTimeout: time.Duration(member.SessionTimeout) * time.Millisecond,
RebalanceTimeout: time.Duration(member.RebalanceTimeout) * time.Millisecond, RebalanceTimeout: time.Duration(member.RebalanceTimeout) * time.Millisecond,
} }
// Calculate time until session timeout // Calculate time until session timeout
sessionTimeRemaining := memberStatus.SessionTimeout - now.Sub(member.LastHeartbeat) sessionTimeRemaining := memberStatus.SessionTimeout - now.Sub(member.LastHeartbeat)
if sessionTimeRemaining < 0 { if sessionTimeRemaining < 0 {
sessionTimeRemaining = 0 sessionTimeRemaining = 0
} }
memberStatus.SessionTimeRemaining = sessionTimeRemaining memberStatus.SessionTimeRemaining = sessionTimeRemaining
// Calculate time until rebalance timeout // Calculate time until rebalance timeout
rebalanceTimeRemaining := memberStatus.RebalanceTimeout - now.Sub(member.JoinedAt) rebalanceTimeRemaining := memberStatus.RebalanceTimeout - now.Sub(member.JoinedAt)
if rebalanceTimeRemaining < 0 { if rebalanceTimeRemaining < 0 {
rebalanceTimeRemaining = 0 rebalanceTimeRemaining = 0
} }
memberStatus.RebalanceTimeRemaining = rebalanceTimeRemaining memberStatus.RebalanceTimeRemaining = rebalanceTimeRemaining
status.Members = append(status.Members, memberStatus) status.Members = append(status.Members, memberStatus)
} }
return status return status
} }
// RebalanceStatus represents the current status of a group's rebalance // RebalanceStatus represents the current status of a group's rebalance
type RebalanceStatus struct { type RebalanceStatus struct {
GroupID string `json:"group_id"`
State GroupState `json:"state"`
Generation int32 `json:"generation"`
MemberCount int `json:"member_count"`
Leader string `json:"leader"`
LastActivity time.Time `json:"last_activity"`
IsRebalancing bool `json:"is_rebalancing"`
RebalanceDuration time.Duration `json:"rebalance_duration"`
Members []MemberTimeoutStatus `json:"members"`
GroupID string `json:"group_id"`
State GroupState `json:"state"`
Generation int32 `json:"generation"`
MemberCount int `json:"member_count"`
Leader string `json:"leader"`
LastActivity time.Time `json:"last_activity"`
IsRebalancing bool `json:"is_rebalancing"`
RebalanceDuration time.Duration `json:"rebalance_duration"`
Members []MemberTimeoutStatus `json:"members"`
} }
// MemberTimeoutStatus represents timeout status for a group member // MemberTimeoutStatus represents timeout status for a group member
type MemberTimeoutStatus struct { type MemberTimeoutStatus struct {
MemberID string `json:"member_id"`
State MemberState `json:"state"`
LastHeartbeat time.Time `json:"last_heartbeat"`
JoinedAt time.Time `json:"joined_at"`
SessionTimeout time.Duration `json:"session_timeout"`
RebalanceTimeout time.Duration `json:"rebalance_timeout"`
SessionTimeRemaining time.Duration `json:"session_time_remaining"`
RebalanceTimeRemaining time.Duration `json:"rebalance_time_remaining"`
MemberID string `json:"member_id"`
State MemberState `json:"state"`
LastHeartbeat time.Time `json:"last_heartbeat"`
JoinedAt time.Time `json:"joined_at"`
SessionTimeout time.Duration `json:"session_timeout"`
RebalanceTimeout time.Duration `json:"rebalance_timeout"`
SessionTimeRemaining time.Duration `json:"session_time_remaining"`
RebalanceTimeRemaining time.Duration `json:"rebalance_time_remaining"`
} }

112
weed/mq/kafka/consumer/rebalance_timeout_test.go

@ -8,14 +8,14 @@ import (
func TestRebalanceTimeoutManager_CheckRebalanceTimeouts(t *testing.T) { func TestRebalanceTimeoutManager_CheckRebalanceTimeouts(t *testing.T) {
coordinator := NewGroupCoordinator() coordinator := NewGroupCoordinator()
defer coordinator.Close() defer coordinator.Close()
rtm := coordinator.rebalanceTimeoutManager rtm := coordinator.rebalanceTimeoutManager
// Create a group with a member that has a short rebalance timeout // Create a group with a member that has a short rebalance timeout
group := coordinator.GetOrCreateGroup("test-group") group := coordinator.GetOrCreateGroup("test-group")
group.Mu.Lock() group.Mu.Lock()
group.State = GroupStatePreparingRebalance group.State = GroupStatePreparingRebalance
member := &GroupMember{ member := &GroupMember{
ID: "member1", ID: "member1",
ClientID: "client1", ClientID: "client1",
@ -27,15 +27,15 @@ func TestRebalanceTimeoutManager_CheckRebalanceTimeouts(t *testing.T) {
} }
group.Members["member1"] = member group.Members["member1"] = member
group.Mu.Unlock() group.Mu.Unlock()
// Check timeouts - member should be evicted // Check timeouts - member should be evicted
rtm.CheckRebalanceTimeouts() rtm.CheckRebalanceTimeouts()
group.Mu.RLock() group.Mu.RLock()
if len(group.Members) != 0 { if len(group.Members) != 0 {
t.Errorf("Expected member to be evicted due to rebalance timeout, but %d members remain", len(group.Members)) t.Errorf("Expected member to be evicted due to rebalance timeout, but %d members remain", len(group.Members))
} }
if group.State != GroupStateEmpty { if group.State != GroupStateEmpty {
t.Errorf("Expected group state to be Empty after member eviction, got %s", group.State.String()) t.Errorf("Expected group state to be Empty after member eviction, got %s", group.State.String())
} }
@ -45,18 +45,18 @@ func TestRebalanceTimeoutManager_CheckRebalanceTimeouts(t *testing.T) {
func TestRebalanceTimeoutManager_SessionTimeoutFallback(t *testing.T) { func TestRebalanceTimeoutManager_SessionTimeoutFallback(t *testing.T) {
coordinator := NewGroupCoordinator() coordinator := NewGroupCoordinator()
defer coordinator.Close() defer coordinator.Close()
rtm := coordinator.rebalanceTimeoutManager rtm := coordinator.rebalanceTimeoutManager
// Create a group with a member that has exceeded session timeout // Create a group with a member that has exceeded session timeout
group := coordinator.GetOrCreateGroup("test-group") group := coordinator.GetOrCreateGroup("test-group")
group.Mu.Lock() group.Mu.Lock()
group.State = GroupStatePreparingRebalance group.State = GroupStatePreparingRebalance
member := &GroupMember{ member := &GroupMember{
ID: "member1", ID: "member1",
ClientID: "client1", ClientID: "client1",
SessionTimeout: 1000, // 1 second
SessionTimeout: 1000, // 1 second
RebalanceTimeout: 30000, // 30 seconds RebalanceTimeout: 30000, // 30 seconds
State: MemberStatePending, State: MemberStatePending,
LastHeartbeat: time.Now().Add(-2 * time.Second), // Last heartbeat 2 seconds ago LastHeartbeat: time.Now().Add(-2 * time.Second), // Last heartbeat 2 seconds ago
@ -64,10 +64,10 @@ func TestRebalanceTimeoutManager_SessionTimeoutFallback(t *testing.T) {
} }
group.Members["member1"] = member group.Members["member1"] = member
group.Mu.Unlock() group.Mu.Unlock()
// Check timeouts - member should be evicted due to session timeout // Check timeouts - member should be evicted due to session timeout
rtm.CheckRebalanceTimeouts() rtm.CheckRebalanceTimeouts()
group.Mu.RLock() group.Mu.RLock()
if len(group.Members) != 0 { if len(group.Members) != 0 {
t.Errorf("Expected member to be evicted due to session timeout, but %d members remain", len(group.Members)) t.Errorf("Expected member to be evicted due to session timeout, but %d members remain", len(group.Members))
@ -78,15 +78,15 @@ func TestRebalanceTimeoutManager_SessionTimeoutFallback(t *testing.T) {
func TestRebalanceTimeoutManager_LeaderEviction(t *testing.T) { func TestRebalanceTimeoutManager_LeaderEviction(t *testing.T) {
coordinator := NewGroupCoordinator() coordinator := NewGroupCoordinator()
defer coordinator.Close() defer coordinator.Close()
rtm := coordinator.rebalanceTimeoutManager rtm := coordinator.rebalanceTimeoutManager
// Create a group with leader and another member // Create a group with leader and another member
group := coordinator.GetOrCreateGroup("test-group") group := coordinator.GetOrCreateGroup("test-group")
group.Mu.Lock() group.Mu.Lock()
group.State = GroupStatePreparingRebalance group.State = GroupStatePreparingRebalance
group.Leader = "member1" group.Leader = "member1"
// Leader with expired rebalance timeout // Leader with expired rebalance timeout
leader := &GroupMember{ leader := &GroupMember{
ID: "member1", ID: "member1",
@ -98,7 +98,7 @@ func TestRebalanceTimeoutManager_LeaderEviction(t *testing.T) {
JoinedAt: time.Now().Add(-2 * time.Second), JoinedAt: time.Now().Add(-2 * time.Second),
} }
group.Members["member1"] = leader group.Members["member1"] = leader
// Another member that's still valid // Another member that's still valid
member2 := &GroupMember{ member2 := &GroupMember{
ID: "member2", ID: "member2",
@ -111,19 +111,19 @@ func TestRebalanceTimeoutManager_LeaderEviction(t *testing.T) {
} }
group.Members["member2"] = member2 group.Members["member2"] = member2
group.Mu.Unlock() group.Mu.Unlock()
// Check timeouts - leader should be evicted, new leader selected // Check timeouts - leader should be evicted, new leader selected
rtm.CheckRebalanceTimeouts() rtm.CheckRebalanceTimeouts()
group.Mu.RLock() group.Mu.RLock()
if len(group.Members) != 1 { if len(group.Members) != 1 {
t.Errorf("Expected 1 member to remain after leader eviction, got %d", len(group.Members)) t.Errorf("Expected 1 member to remain after leader eviction, got %d", len(group.Members))
} }
if group.Leader != "member2" { if group.Leader != "member2" {
t.Errorf("Expected member2 to become new leader, got %s", group.Leader) t.Errorf("Expected member2 to become new leader, got %s", group.Leader)
} }
if group.State != GroupStatePreparingRebalance { if group.State != GroupStatePreparingRebalance {
t.Errorf("Expected group to restart rebalancing after leader eviction, got %s", group.State.String()) t.Errorf("Expected group to restart rebalancing after leader eviction, got %s", group.State.String())
} }
@ -133,37 +133,37 @@ func TestRebalanceTimeoutManager_LeaderEviction(t *testing.T) {
func TestRebalanceTimeoutManager_IsRebalanceStuck(t *testing.T) { func TestRebalanceTimeoutManager_IsRebalanceStuck(t *testing.T) {
coordinator := NewGroupCoordinator() coordinator := NewGroupCoordinator()
defer coordinator.Close() defer coordinator.Close()
rtm := coordinator.rebalanceTimeoutManager rtm := coordinator.rebalanceTimeoutManager
// Create a group that's been rebalancing for a while // Create a group that's been rebalancing for a while
group := coordinator.GetOrCreateGroup("test-group") group := coordinator.GetOrCreateGroup("test-group")
group.Mu.Lock() group.Mu.Lock()
group.State = GroupStatePreparingRebalance group.State = GroupStatePreparingRebalance
group.LastActivity = time.Now().Add(-15 * time.Minute) // 15 minutes ago group.LastActivity = time.Now().Add(-15 * time.Minute) // 15 minutes ago
group.Mu.Unlock() group.Mu.Unlock()
// Check if rebalance is stuck (max 10 minutes) // Check if rebalance is stuck (max 10 minutes)
maxDuration := 10 * time.Minute maxDuration := 10 * time.Minute
if !rtm.IsRebalanceStuck(group, maxDuration) { if !rtm.IsRebalanceStuck(group, maxDuration) {
t.Error("Expected rebalance to be detected as stuck") t.Error("Expected rebalance to be detected as stuck")
} }
// Test with a group that's not stuck // Test with a group that's not stuck
group.Mu.Lock() group.Mu.Lock()
group.LastActivity = time.Now().Add(-5 * time.Minute) // 5 minutes ago group.LastActivity = time.Now().Add(-5 * time.Minute) // 5 minutes ago
group.Mu.Unlock() group.Mu.Unlock()
if rtm.IsRebalanceStuck(group, maxDuration) { if rtm.IsRebalanceStuck(group, maxDuration) {
t.Error("Expected rebalance to not be detected as stuck") t.Error("Expected rebalance to not be detected as stuck")
} }
// Test with stable group (should not be stuck) // Test with stable group (should not be stuck)
group.Mu.Lock() group.Mu.Lock()
group.State = GroupStateStable group.State = GroupStateStable
group.LastActivity = time.Now().Add(-15 * time.Minute) group.LastActivity = time.Now().Add(-15 * time.Minute)
group.Mu.Unlock() group.Mu.Unlock()
if rtm.IsRebalanceStuck(group, maxDuration) { if rtm.IsRebalanceStuck(group, maxDuration) {
t.Error("Stable group should not be detected as stuck") t.Error("Stable group should not be detected as stuck")
} }
@ -172,37 +172,37 @@ func TestRebalanceTimeoutManager_IsRebalanceStuck(t *testing.T) {
func TestRebalanceTimeoutManager_ForceCompleteRebalance(t *testing.T) { func TestRebalanceTimeoutManager_ForceCompleteRebalance(t *testing.T) {
coordinator := NewGroupCoordinator() coordinator := NewGroupCoordinator()
defer coordinator.Close() defer coordinator.Close()
rtm := coordinator.rebalanceTimeoutManager rtm := coordinator.rebalanceTimeoutManager
// Test forcing completion from PreparingRebalance // Test forcing completion from PreparingRebalance
group := coordinator.GetOrCreateGroup("test-group") group := coordinator.GetOrCreateGroup("test-group")
group.Mu.Lock() group.Mu.Lock()
group.State = GroupStatePreparingRebalance group.State = GroupStatePreparingRebalance
member := &GroupMember{ member := &GroupMember{
ID: "member1", ID: "member1",
State: MemberStatePending, State: MemberStatePending,
} }
group.Members["member1"] = member group.Members["member1"] = member
group.Mu.Unlock() group.Mu.Unlock()
rtm.ForceCompleteRebalance(group) rtm.ForceCompleteRebalance(group)
group.Mu.RLock() group.Mu.RLock()
if group.State != GroupStateCompletingRebalance { if group.State != GroupStateCompletingRebalance {
t.Errorf("Expected group state to be CompletingRebalance, got %s", group.State.String()) t.Errorf("Expected group state to be CompletingRebalance, got %s", group.State.String())
} }
group.Mu.RUnlock() group.Mu.RUnlock()
// Test forcing completion from CompletingRebalance // Test forcing completion from CompletingRebalance
rtm.ForceCompleteRebalance(group) rtm.ForceCompleteRebalance(group)
group.Mu.RLock() group.Mu.RLock()
if group.State != GroupStateStable { if group.State != GroupStateStable {
t.Errorf("Expected group state to be Stable, got %s", group.State.String()) t.Errorf("Expected group state to be Stable, got %s", group.State.String())
} }
if member.State != MemberStateStable { if member.State != MemberStateStable {
t.Errorf("Expected member state to be Stable, got %s", member.State.String()) t.Errorf("Expected member state to be Stable, got %s", member.State.String())
} }
@ -212,15 +212,15 @@ func TestRebalanceTimeoutManager_ForceCompleteRebalance(t *testing.T) {
func TestRebalanceTimeoutManager_GetRebalanceStatus(t *testing.T) { func TestRebalanceTimeoutManager_GetRebalanceStatus(t *testing.T) {
coordinator := NewGroupCoordinator() coordinator := NewGroupCoordinator()
defer coordinator.Close() defer coordinator.Close()
rtm := coordinator.rebalanceTimeoutManager rtm := coordinator.rebalanceTimeoutManager
// Test with non-existent group // Test with non-existent group
status := rtm.GetRebalanceStatus("non-existent") status := rtm.GetRebalanceStatus("non-existent")
if status != nil { if status != nil {
t.Error("Expected nil status for non-existent group") t.Error("Expected nil status for non-existent group")
} }
// Create a group with members // Create a group with members
group := coordinator.GetOrCreateGroup("test-group") group := coordinator.GetOrCreateGroup("test-group")
group.Mu.Lock() group.Mu.Lock()
@ -228,7 +228,7 @@ func TestRebalanceTimeoutManager_GetRebalanceStatus(t *testing.T) {
group.Generation = 5 group.Generation = 5
group.Leader = "member1" group.Leader = "member1"
group.LastActivity = time.Now().Add(-2 * time.Minute) group.LastActivity = time.Now().Add(-2 * time.Minute)
member1 := &GroupMember{ member1 := &GroupMember{
ID: "member1", ID: "member1",
State: MemberStatePending, State: MemberStatePending,
@ -238,7 +238,7 @@ func TestRebalanceTimeoutManager_GetRebalanceStatus(t *testing.T) {
RebalanceTimeout: 300000, // 5 minutes RebalanceTimeout: 300000, // 5 minutes
} }
group.Members["member1"] = member1 group.Members["member1"] = member1
member2 := &GroupMember{ member2 := &GroupMember{
ID: "member2", ID: "member2",
State: MemberStatePending, State: MemberStatePending,
@ -249,48 +249,48 @@ func TestRebalanceTimeoutManager_GetRebalanceStatus(t *testing.T) {
} }
group.Members["member2"] = member2 group.Members["member2"] = member2
group.Mu.Unlock() group.Mu.Unlock()
// Get status // Get status
status = rtm.GetRebalanceStatus("test-group") status = rtm.GetRebalanceStatus("test-group")
if status == nil { if status == nil {
t.Fatal("Expected non-nil status") t.Fatal("Expected non-nil status")
} }
if status.GroupID != "test-group" { if status.GroupID != "test-group" {
t.Errorf("Expected group ID 'test-group', got %s", status.GroupID) t.Errorf("Expected group ID 'test-group', got %s", status.GroupID)
} }
if status.State != GroupStatePreparingRebalance { if status.State != GroupStatePreparingRebalance {
t.Errorf("Expected state PreparingRebalance, got %s", status.State.String()) t.Errorf("Expected state PreparingRebalance, got %s", status.State.String())
} }
if status.Generation != 5 { if status.Generation != 5 {
t.Errorf("Expected generation 5, got %d", status.Generation) t.Errorf("Expected generation 5, got %d", status.Generation)
} }
if status.MemberCount != 2 { if status.MemberCount != 2 {
t.Errorf("Expected 2 members, got %d", status.MemberCount) t.Errorf("Expected 2 members, got %d", status.MemberCount)
} }
if status.Leader != "member1" { if status.Leader != "member1" {
t.Errorf("Expected leader 'member1', got %s", status.Leader) t.Errorf("Expected leader 'member1', got %s", status.Leader)
} }
if !status.IsRebalancing { if !status.IsRebalancing {
t.Error("Expected IsRebalancing to be true") t.Error("Expected IsRebalancing to be true")
} }
if len(status.Members) != 2 { if len(status.Members) != 2 {
t.Errorf("Expected 2 member statuses, got %d", len(status.Members)) t.Errorf("Expected 2 member statuses, got %d", len(status.Members))
} }
// Check member timeout calculations // Check member timeout calculations
for _, memberStatus := range status.Members { for _, memberStatus := range status.Members {
if memberStatus.SessionTimeRemaining < 0 { if memberStatus.SessionTimeRemaining < 0 {
t.Errorf("Session time remaining should not be negative for member %s", memberStatus.MemberID) t.Errorf("Session time remaining should not be negative for member %s", memberStatus.MemberID)
} }
if memberStatus.RebalanceTimeRemaining < 0 { if memberStatus.RebalanceTimeRemaining < 0 {
t.Errorf("Rebalance time remaining should not be negative for member %s", memberStatus.MemberID) t.Errorf("Rebalance time remaining should not be negative for member %s", memberStatus.MemberID)
} }
@ -300,14 +300,14 @@ func TestRebalanceTimeoutManager_GetRebalanceStatus(t *testing.T) {
func TestRebalanceTimeoutManager_DefaultRebalanceTimeout(t *testing.T) { func TestRebalanceTimeoutManager_DefaultRebalanceTimeout(t *testing.T) {
coordinator := NewGroupCoordinator() coordinator := NewGroupCoordinator()
defer coordinator.Close() defer coordinator.Close()
rtm := coordinator.rebalanceTimeoutManager rtm := coordinator.rebalanceTimeoutManager
// Create a group with a member that has no rebalance timeout set (0) // Create a group with a member that has no rebalance timeout set (0)
group := coordinator.GetOrCreateGroup("test-group") group := coordinator.GetOrCreateGroup("test-group")
group.Mu.Lock() group.Mu.Lock()
group.State = GroupStatePreparingRebalance group.State = GroupStatePreparingRebalance
member := &GroupMember{ member := &GroupMember{
ID: "member1", ID: "member1",
ClientID: "client1", ClientID: "client1",
@ -319,10 +319,10 @@ func TestRebalanceTimeoutManager_DefaultRebalanceTimeout(t *testing.T) {
} }
group.Members["member1"] = member group.Members["member1"] = member
group.Mu.Unlock() group.Mu.Unlock()
// Default rebalance timeout is 5 minutes (300000ms), so member should be evicted // Default rebalance timeout is 5 minutes (300000ms), so member should be evicted
rtm.CheckRebalanceTimeouts() rtm.CheckRebalanceTimeouts()
group.Mu.RLock() group.Mu.RLock()
if len(group.Members) != 0 { if len(group.Members) != 0 {
t.Errorf("Expected member to be evicted using default rebalance timeout, but %d members remain", len(group.Members)) t.Errorf("Expected member to be evicted using default rebalance timeout, but %d members remain", len(group.Members))

1
weed/mq/kafka/consumer_offset/memory_storage.go

@ -142,4 +142,3 @@ func (m *MemoryStorage) Close() error {
return nil return nil
} }

1
weed/mq/kafka/consumer_offset/memory_storage_test.go

@ -206,4 +206,3 @@ func TestMemoryStorageOverwrite(t *testing.T) {
assert.Equal(t, int64(20), offset) assert.Equal(t, int64(20), offset)
assert.Equal(t, "meta2", metadata) assert.Equal(t, "meta2", metadata)
} }

1
weed/mq/kafka/consumer_offset/storage.go

@ -56,4 +56,3 @@ var (
ErrInvalidPartition = fmt.Errorf("invalid partition") ErrInvalidPartition = fmt.Errorf("invalid partition")
ErrStorageClosed = fmt.Errorf("storage is closed") ErrStorageClosed = fmt.Errorf("storage is closed")
) )

1
weed/mq/kafka/gateway/test_mock_handler.go

@ -121,7 +121,6 @@ func (m *mockSeaweedMQHandler) ProduceRecord(ctx context.Context, topicName stri
offset := m.offsets[topicName][partitionID] offset := m.offsets[topicName][partitionID]
m.offsets[topicName][partitionID]++ m.offsets[topicName][partitionID]++
// Store record // Store record
record := &mockRecord{ record := &mockRecord{
key: key, key: key,

2
weed/mq/kafka/package.go

@ -9,5 +9,3 @@ package kafka
// - offset/: Offset management // - offset/: Offset management
// - schema/: Schema registry integration // - schema/: Schema registry integration
// - consumer/: Consumer group coordination // - consumer/: Consumer group coordination

2
weed/mq/kafka/partition_mapping.go

@ -51,5 +51,3 @@ func GetRangeSize() int32 {
func GetMaxKafkaPartitions() int32 { func GetMaxKafkaPartitions() int32 {
return int32(pub_balancer.MaxPartitionCount) / 35 // 72 partitions return int32(pub_balancer.MaxPartitionCount) / 35 // 72 partitions
} }

2
weed/mq/kafka/protocol/describe_cluster.go

@ -37,7 +37,6 @@ func (h *Handler) handleDescribeCluster(correlationID uint32, apiVersion uint16,
// Tagged fields at end of request // Tagged fields at end of request
// (We don't parse them, just skip) // (We don't parse them, just skip)
// Build response // Build response
response := make([]byte, 0, 256) response := make([]byte, 0, 256)
@ -109,6 +108,5 @@ func (h *Handler) handleDescribeCluster(correlationID uint32, apiVersion uint16,
// Response-level tagged fields (flexible response) // Response-level tagged fields (flexible response)
response = append(response, 0x00) // Empty tagged fields response = append(response, 0x00) // Empty tagged fields
return response, nil return response, nil
} }

1
weed/mq/kafka/protocol/flexible_versions.go

@ -268,7 +268,6 @@ func parseCompactString(data []byte) ([]byte, int) {
return nil, 0 return nil, 0
} }
if actualLength == 0 { if actualLength == 0 {
// Empty string (length was 1) // Empty string (length was 1)
return []byte{}, consumed return []byte{}, consumed

18
weed/mq/kafka/protocol/group_introspection.go

@ -107,13 +107,13 @@ func (h *Handler) describeGroup(groupID string) DescribeGroupsGroup {
} }
return DescribeGroupsGroup{ return DescribeGroupsGroup{
ErrorCode: 0,
GroupID: groupID,
State: stateStr,
ProtocolType: "consumer", // Default protocol type
Protocol: group.Protocol,
Members: members,
AuthorizedOps: []int32{}, // Empty for now
ErrorCode: 0,
GroupID: groupID,
State: stateStr,
ProtocolType: "consumer", // Default protocol type
Protocol: group.Protocol,
Members: members,
AuthorizedOps: []int32{}, // Empty for now
} }
} }
@ -175,8 +175,8 @@ func (h *Handler) listAllGroups(statesFilter []string) []ListGroupsGroup {
// Request/Response structures // Request/Response structures
type DescribeGroupsRequest struct { type DescribeGroupsRequest struct {
GroupIDs []string
IncludeAuthorizedOps bool
GroupIDs []string
IncludeAuthorizedOps bool
} }
type DescribeGroupsResponse struct { type DescribeGroupsResponse struct {

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

@ -661,7 +661,7 @@ func (h *Handler) HandleConn(ctx context.Context, conn net.Conn) error {
return return
} }
// Removed V(4) logging from hot path - only log errors and important events // Removed V(4) logging from hot path - only log errors and important events
// Wrap request processing with panic recovery to prevent deadlocks // Wrap request processing with panic recovery to prevent deadlocks
// If processRequestSync panics, we MUST still send a response to avoid blocking the response writer // If processRequestSync panics, we MUST still send a response to avoid blocking the response writer
var response []byte var response []byte
@ -881,7 +881,6 @@ func (h *Handler) HandleConn(ctx context.Context, conn net.Conn) error {
return fmt.Errorf("read message: %w", err) return fmt.Errorf("read message: %w", err)
} }
// Parse at least the basic header to get API key and correlation ID // Parse at least the basic header to get API key and correlation ID
if len(messageBuf) < 8 { if len(messageBuf) < 8 {
return fmt.Errorf("message too short") return fmt.Errorf("message too short")
@ -890,7 +889,7 @@ func (h *Handler) HandleConn(ctx context.Context, conn net.Conn) error {
apiKey := binary.BigEndian.Uint16(messageBuf[0:2]) apiKey := binary.BigEndian.Uint16(messageBuf[0:2])
apiVersion := binary.BigEndian.Uint16(messageBuf[2:4]) apiVersion := binary.BigEndian.Uint16(messageBuf[2:4])
correlationID := binary.BigEndian.Uint32(messageBuf[4:8]) correlationID := binary.BigEndian.Uint32(messageBuf[4:8])
// Validate API version against what we support // Validate API version against what we support
if err := h.validateAPIVersion(apiKey, apiVersion); err != nil { if err := h.validateAPIVersion(apiKey, apiVersion); err != nil {
glog.Errorf("API VERSION VALIDATION FAILED: Key=%d (%s), Version=%d, error=%v", apiKey, getAPIName(APIKey(apiKey)), apiVersion, err) glog.Errorf("API VERSION VALIDATION FAILED: Key=%d (%s), Version=%d, error=%v", apiKey, getAPIName(APIKey(apiKey)), apiVersion, err)
@ -1050,7 +1049,6 @@ func (h *Handler) processRequestSync(req *kafkaRequest) ([]byte, error) {
requestStart := time.Now() requestStart := time.Now()
apiName := getAPIName(APIKey(req.apiKey)) apiName := getAPIName(APIKey(req.apiKey))
// Only log high-volume requests at V(2), not V(4) // Only log high-volume requests at V(2), not V(4)
if glog.V(2) { if glog.V(2) {
glog.V(2).Infof("[API] %s (key=%d, ver=%d, corr=%d)", glog.V(2).Infof("[API] %s (key=%d, ver=%d, corr=%d)",
@ -1589,15 +1587,15 @@ func (h *Handler) HandleMetadataV2(correlationID uint32, requestBody []byte) ([]
for partitionID := int32(0); partitionID < partitionCount; partitionID++ { for partitionID := int32(0); partitionID < partitionCount; partitionID++ {
binary.Write(&buf, binary.BigEndian, int16(0)) // ErrorCode binary.Write(&buf, binary.BigEndian, int16(0)) // ErrorCode
binary.Write(&buf, binary.BigEndian, partitionID) // PartitionIndex binary.Write(&buf, binary.BigEndian, partitionID) // PartitionIndex
binary.Write(&buf, binary.BigEndian, nodeID) // LeaderID
binary.Write(&buf, binary.BigEndian, nodeID) // LeaderID
// ReplicaNodes array (4 bytes length + nodes) // ReplicaNodes array (4 bytes length + nodes)
binary.Write(&buf, binary.BigEndian, int32(1)) // 1 replica binary.Write(&buf, binary.BigEndian, int32(1)) // 1 replica
binary.Write(&buf, binary.BigEndian, nodeID) // NodeID 1
binary.Write(&buf, binary.BigEndian, nodeID) // NodeID 1
// IsrNodes array (4 bytes length + nodes) // IsrNodes array (4 bytes length + nodes)
binary.Write(&buf, binary.BigEndian, int32(1)) // 1 ISR node binary.Write(&buf, binary.BigEndian, int32(1)) // 1 ISR node
binary.Write(&buf, binary.BigEndian, nodeID) // NodeID 1
binary.Write(&buf, binary.BigEndian, nodeID) // NodeID 1
} }
} }
@ -1716,15 +1714,15 @@ func (h *Handler) HandleMetadataV3V4(correlationID uint32, requestBody []byte) (
for partitionID := int32(0); partitionID < partitionCount; partitionID++ { for partitionID := int32(0); partitionID < partitionCount; partitionID++ {
binary.Write(&buf, binary.BigEndian, int16(0)) // ErrorCode binary.Write(&buf, binary.BigEndian, int16(0)) // ErrorCode
binary.Write(&buf, binary.BigEndian, partitionID) // PartitionIndex binary.Write(&buf, binary.BigEndian, partitionID) // PartitionIndex
binary.Write(&buf, binary.BigEndian, nodeID) // LeaderID
binary.Write(&buf, binary.BigEndian, nodeID) // LeaderID
// ReplicaNodes array (4 bytes length + nodes) // ReplicaNodes array (4 bytes length + nodes)
binary.Write(&buf, binary.BigEndian, int32(1)) // 1 replica binary.Write(&buf, binary.BigEndian, int32(1)) // 1 replica
binary.Write(&buf, binary.BigEndian, nodeID) // NodeID 1
binary.Write(&buf, binary.BigEndian, nodeID) // NodeID 1
// IsrNodes array (4 bytes length + nodes) // IsrNodes array (4 bytes length + nodes)
binary.Write(&buf, binary.BigEndian, int32(1)) // 1 ISR node binary.Write(&buf, binary.BigEndian, int32(1)) // 1 ISR node
binary.Write(&buf, binary.BigEndian, nodeID) // NodeID 1
binary.Write(&buf, binary.BigEndian, nodeID) // NodeID 1
} }
} }
@ -1737,7 +1735,7 @@ func (h *Handler) HandleMetadataV3V4(correlationID uint32, requestBody []byte) (
} }
if len(response) > 100 { if len(response) > 100 {
} }
return response, nil return response, nil
} }
@ -1828,7 +1826,6 @@ func (h *Handler) handleMetadataV5ToV8(correlationID uint32, requestBody []byte,
// NOTE: Correlation ID is handled by writeResponseWithCorrelationID // NOTE: Correlation ID is handled by writeResponseWithCorrelationID
// Do NOT include it in the response body // Do NOT include it in the response body
// ThrottleTimeMs (4 bytes) - v3+ addition // ThrottleTimeMs (4 bytes) - v3+ addition
binary.Write(&buf, binary.BigEndian, int32(0)) // No throttling binary.Write(&buf, binary.BigEndian, int32(0)) // No throttling
@ -1896,7 +1893,7 @@ func (h *Handler) handleMetadataV5ToV8(correlationID uint32, requestBody []byte,
for partitionID := int32(0); partitionID < partitionCount; partitionID++ { for partitionID := int32(0); partitionID < partitionCount; partitionID++ {
binary.Write(&buf, binary.BigEndian, int16(0)) // ErrorCode binary.Write(&buf, binary.BigEndian, int16(0)) // ErrorCode
binary.Write(&buf, binary.BigEndian, partitionID) // PartitionIndex binary.Write(&buf, binary.BigEndian, partitionID) // PartitionIndex
binary.Write(&buf, binary.BigEndian, nodeID) // LeaderID
binary.Write(&buf, binary.BigEndian, nodeID) // LeaderID
// LeaderEpoch (4 bytes) - v7+ addition // LeaderEpoch (4 bytes) - v7+ addition
if apiVersion >= 7 { if apiVersion >= 7 {
@ -1905,11 +1902,11 @@ func (h *Handler) handleMetadataV5ToV8(correlationID uint32, requestBody []byte,
// ReplicaNodes array (4 bytes length + nodes) // ReplicaNodes array (4 bytes length + nodes)
binary.Write(&buf, binary.BigEndian, int32(1)) // 1 replica binary.Write(&buf, binary.BigEndian, int32(1)) // 1 replica
binary.Write(&buf, binary.BigEndian, nodeID) // NodeID 1
binary.Write(&buf, binary.BigEndian, nodeID) // NodeID 1
// IsrNodes array (4 bytes length + nodes) // IsrNodes array (4 bytes length + nodes)
binary.Write(&buf, binary.BigEndian, int32(1)) // 1 ISR node binary.Write(&buf, binary.BigEndian, int32(1)) // 1 ISR node
binary.Write(&buf, binary.BigEndian, nodeID) // NodeID 1
binary.Write(&buf, binary.BigEndian, nodeID) // NodeID 1
// OfflineReplicas array (4 bytes length + nodes) - v5+ addition // OfflineReplicas array (4 bytes length + nodes) - v5+ addition
binary.Write(&buf, binary.BigEndian, int32(0)) // No offline replicas binary.Write(&buf, binary.BigEndian, int32(0)) // No offline replicas
@ -1930,7 +1927,7 @@ func (h *Handler) handleMetadataV5ToV8(correlationID uint32, requestBody []byte,
} }
if len(response) > 100 { if len(response) > 100 {
} }
return response, nil return response, nil
} }
@ -1994,12 +1991,11 @@ func (h *Handler) handleListOffsets(correlationID uint32, apiVersion uint16, req
// Parse minimal request to understand what's being asked (header already stripped) // Parse minimal request to understand what's being asked (header already stripped)
offset := 0 offset := 0
maxBytes := len(requestBody) maxBytes := len(requestBody)
if maxBytes > 64 { if maxBytes > 64 {
maxBytes = 64 maxBytes = 64
} }
// v1+ has replica_id(4) // v1+ has replica_id(4)
if apiVersion >= 1 { if apiVersion >= 1 {
if len(requestBody) < offset+4 { if len(requestBody) < offset+4 {
@ -3930,12 +3926,11 @@ func (h *Handler) handleInitProducerId(correlationID uint32, apiVersion uint16,
// v2+: transactional_id(NULLABLE_STRING) + transaction_timeout_ms(INT32) + producer_id(INT64) + producer_epoch(INT16) // v2+: transactional_id(NULLABLE_STRING) + transaction_timeout_ms(INT32) + producer_id(INT64) + producer_epoch(INT16)
// v4+: Uses flexible format with tagged fields // v4+: Uses flexible format with tagged fields
maxBytes := len(requestBody) maxBytes := len(requestBody)
if maxBytes > 64 { if maxBytes > 64 {
maxBytes = 64 maxBytes = 64
} }
offset := 0 offset := 0
// Parse transactional_id (NULLABLE_STRING or COMPACT_NULLABLE_STRING for flexible versions) // Parse transactional_id (NULLABLE_STRING or COMPACT_NULLABLE_STRING for flexible versions)

1
weed/mq/kafka/protocol/offset_storage_adapter.go

@ -47,4 +47,3 @@ func (a *offsetStorageAdapter) DeleteGroup(group string) error {
func (a *offsetStorageAdapter) Close() error { func (a *offsetStorageAdapter) Close() error {
return a.storage.Close() return a.storage.Close()
} }

1
weed/mq/kafka/protocol/response_validation_example_test.go

@ -140,4 +140,3 @@ func TestMetadataResponseHasBrokers(t *testing.T) {
t.Logf("✓ Metadata response correctly has %d broker(s)", parsedCount) t.Logf("✓ Metadata response correctly has %d broker(s)", parsedCount)
} }

72
weed/mq/kafka/schema/envelope_test.go

@ -7,46 +7,46 @@ import (
func TestParseConfluentEnvelope(t *testing.T) { func TestParseConfluentEnvelope(t *testing.T) {
tests := []struct { tests := []struct {
name string
input []byte
expectOK bool
expectID uint32
name string
input []byte
expectOK bool
expectID uint32
expectFormat Format expectFormat Format
}{ }{
{ {
name: "valid Avro message",
input: []byte{0x00, 0x00, 0x00, 0x00, 0x01, 0x10, 0x48, 0x65, 0x6c, 0x6c, 0x6f}, // schema ID 1 + "Hello"
expectOK: true,
expectID: 1,
name: "valid Avro message",
input: []byte{0x00, 0x00, 0x00, 0x00, 0x01, 0x10, 0x48, 0x65, 0x6c, 0x6c, 0x6f}, // schema ID 1 + "Hello"
expectOK: true,
expectID: 1,
expectFormat: FormatAvro, expectFormat: FormatAvro,
}, },
{ {
name: "valid message with larger schema ID",
input: []byte{0x00, 0x00, 0x00, 0x04, 0xd2, 0x02, 0x66, 0x6f, 0x6f}, // schema ID 1234 + "foo"
expectOK: true,
expectID: 1234,
name: "valid message with larger schema ID",
input: []byte{0x00, 0x00, 0x00, 0x04, 0xd2, 0x02, 0x66, 0x6f, 0x6f}, // schema ID 1234 + "foo"
expectOK: true,
expectID: 1234,
expectFormat: FormatAvro, expectFormat: FormatAvro,
}, },
{ {
name: "too short message",
input: []byte{0x00, 0x00, 0x00},
expectOK: false,
name: "too short message",
input: []byte{0x00, 0x00, 0x00},
expectOK: false,
}, },
{ {
name: "no magic byte",
input: []byte{0x01, 0x00, 0x00, 0x00, 0x01, 0x48, 0x65, 0x6c, 0x6c, 0x6f},
expectOK: false,
name: "no magic byte",
input: []byte{0x01, 0x00, 0x00, 0x00, 0x01, 0x48, 0x65, 0x6c, 0x6c, 0x6f},
expectOK: false,
}, },
{ {
name: "empty message",
input: []byte{},
expectOK: false,
name: "empty message",
input: []byte{},
expectOK: false,
}, },
{ {
name: "minimal valid message",
input: []byte{0x00, 0x00, 0x00, 0x00, 0x01}, // schema ID 1, empty payload
expectOK: true,
expectID: 1,
name: "minimal valid message",
input: []byte{0x00, 0x00, 0x00, 0x00, 0x01}, // schema ID 1, empty payload
expectOK: true,
expectID: 1,
expectFormat: FormatAvro, expectFormat: FormatAvro,
}, },
} }
@ -54,24 +54,24 @@ func TestParseConfluentEnvelope(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
envelope, ok := ParseConfluentEnvelope(tt.input) envelope, ok := ParseConfluentEnvelope(tt.input)
if ok != tt.expectOK { if ok != tt.expectOK {
t.Errorf("ParseConfluentEnvelope() ok = %v, want %v", ok, tt.expectOK) t.Errorf("ParseConfluentEnvelope() ok = %v, want %v", ok, tt.expectOK)
return return
} }
if !tt.expectOK { if !tt.expectOK {
return // No need to check further if we expected failure return // No need to check further if we expected failure
} }
if envelope.SchemaID != tt.expectID { if envelope.SchemaID != tt.expectID {
t.Errorf("ParseConfluentEnvelope() schemaID = %v, want %v", envelope.SchemaID, tt.expectID) t.Errorf("ParseConfluentEnvelope() schemaID = %v, want %v", envelope.SchemaID, tt.expectID)
} }
if envelope.Format != tt.expectFormat { if envelope.Format != tt.expectFormat {
t.Errorf("ParseConfluentEnvelope() format = %v, want %v", envelope.Format, tt.expectFormat) t.Errorf("ParseConfluentEnvelope() format = %v, want %v", envelope.Format, tt.expectFormat)
} }
// Verify payload extraction // Verify payload extraction
expectedPayloadLen := len(tt.input) - 5 // 5 bytes for magic + schema ID expectedPayloadLen := len(tt.input) - 5 // 5 bytes for magic + schema ID
if len(envelope.Payload) != expectedPayloadLen { if len(envelope.Payload) != expectedPayloadLen {
@ -150,11 +150,11 @@ func TestExtractSchemaID(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
id, ok := ExtractSchemaID(tt.input) id, ok := ExtractSchemaID(tt.input)
if ok != tt.expectOK { if ok != tt.expectOK {
t.Errorf("ExtractSchemaID() ok = %v, want %v", ok, tt.expectOK) t.Errorf("ExtractSchemaID() ok = %v, want %v", ok, tt.expectOK)
} }
if id != tt.expectID { if id != tt.expectID {
t.Errorf("ExtractSchemaID() id = %v, want %v", id, tt.expectID) t.Errorf("ExtractSchemaID() id = %v, want %v", id, tt.expectID)
} }
@ -200,12 +200,12 @@ func TestCreateConfluentEnvelope(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
result := CreateConfluentEnvelope(tt.format, tt.schemaID, tt.indexes, tt.payload) result := CreateConfluentEnvelope(tt.format, tt.schemaID, tt.indexes, tt.payload)
if len(result) != len(tt.expected) { if len(result) != len(tt.expected) {
t.Errorf("CreateConfluentEnvelope() length = %v, want %v", len(result), len(tt.expected)) t.Errorf("CreateConfluentEnvelope() length = %v, want %v", len(result), len(tt.expected))
return return
} }
for i, b := range result { for i, b := range result {
if b != tt.expected[i] { if b != tt.expected[i] {
t.Errorf("CreateConfluentEnvelope() byte[%d] = %v, want %v", i, b, tt.expected[i]) t.Errorf("CreateConfluentEnvelope() byte[%d] = %v, want %v", i, b, tt.expected[i])
@ -262,7 +262,7 @@ func TestEnvelopeValidate(t *testing.T) {
for _, tt := range tests { for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) { t.Run(tt.name, func(t *testing.T) {
err := tt.envelope.Validate() err := tt.envelope.Validate()
if (err != nil) != tt.expectErr { if (err != nil) != tt.expectErr {
t.Errorf("Envelope.Validate() error = %v, expectErr %v", err, tt.expectErr) t.Errorf("Envelope.Validate() error = %v, expectErr %v", err, tt.expectErr)
} }
@ -297,7 +297,7 @@ func TestEnvelopeMetadata(t *testing.T) {
func BenchmarkParseConfluentEnvelope(b *testing.B) { func BenchmarkParseConfluentEnvelope(b *testing.B) {
// Create a test message // Create a test message
testMsg := make([]byte, 1024) testMsg := make([]byte, 1024)
testMsg[0] = 0x00 // Magic byte
testMsg[0] = 0x00 // Magic byte
binary.BigEndian.PutUint32(testMsg[1:5], 123) // Schema ID binary.BigEndian.PutUint32(testMsg[1:5], 123) // Schema ID
// Fill rest with dummy data // Fill rest with dummy data
for i := 5; i < len(testMsg); i++ { for i := 5; i < len(testMsg); i++ {

2
weed/mq/kafka/schema/envelope_varint_test.go

@ -100,7 +100,7 @@ func TestCreateConfluentEnvelopeWithProtobufIndexes(t *testing.T) {
parsed, ok := ParseConfluentEnvelope(envelope) parsed, ok := ParseConfluentEnvelope(envelope)
require.True(t, ok, "Should be able to parse envelope") require.True(t, ok, "Should be able to parse envelope")
assert.Equal(t, tc.schemaID, parsed.SchemaID) assert.Equal(t, tc.schemaID, parsed.SchemaID)
if tc.format == FormatProtobuf && len(tc.indexes) == 0 { if tc.format == FormatProtobuf && len(tc.indexes) == 0 {
// For Protobuf without indexes, payload should match // For Protobuf without indexes, payload should match
assert.Equal(t, tc.payload, parsed.Payload, "Payload should match") assert.Equal(t, tc.payload, parsed.Payload, "Payload should match")

2
weed/mq/metadata_constants.go

@ -17,5 +17,3 @@ const (
// Source file tracking for parquet deduplication // Source file tracking for parquet deduplication
ExtendedAttrSources = "sources" // JSON-encoded list of source log files ExtendedAttrSources = "sources" // JSON-encoded list of source log files
) )

54
weed/mq/offset/migration.go

@ -118,17 +118,17 @@ func (m *MigrationManager) GetCurrentVersion() (int, error) {
if err != nil { if err != nil {
return 0, fmt.Errorf("failed to create migrations table: %w", err) return 0, fmt.Errorf("failed to create migrations table: %w", err)
} }
var version sql.NullInt64 var version sql.NullInt64
err = m.db.QueryRow("SELECT MAX(version) FROM schema_migrations").Scan(&version) err = m.db.QueryRow("SELECT MAX(version) FROM schema_migrations").Scan(&version)
if err != nil { if err != nil {
return 0, fmt.Errorf("failed to get current version: %w", err) return 0, fmt.Errorf("failed to get current version: %w", err)
} }
if !version.Valid { if !version.Valid {
return 0, nil // No migrations applied yet return 0, nil // No migrations applied yet
} }
return int(version.Int64), nil return int(version.Int64), nil
} }
@ -138,29 +138,29 @@ func (m *MigrationManager) ApplyMigrations() error {
if err != nil { if err != nil {
return fmt.Errorf("failed to get current version: %w", err) return fmt.Errorf("failed to get current version: %w", err)
} }
migrations := GetMigrations() migrations := GetMigrations()
for _, migration := range migrations { for _, migration := range migrations {
if migration.Version <= currentVersion { if migration.Version <= currentVersion {
continue // Already applied continue // Already applied
} }
fmt.Printf("Applying migration %d: %s\n", migration.Version, migration.Description) fmt.Printf("Applying migration %d: %s\n", migration.Version, migration.Description)
// Begin transaction // Begin transaction
tx, err := m.db.Begin() tx, err := m.db.Begin()
if err != nil { if err != nil {
return fmt.Errorf("failed to begin transaction for migration %d: %w", migration.Version, err) return fmt.Errorf("failed to begin transaction for migration %d: %w", migration.Version, err)
} }
// Execute migration SQL // Execute migration SQL
_, err = tx.Exec(migration.SQL) _, err = tx.Exec(migration.SQL)
if err != nil { if err != nil {
tx.Rollback() tx.Rollback()
return fmt.Errorf("failed to execute migration %d: %w", migration.Version, err) return fmt.Errorf("failed to execute migration %d: %w", migration.Version, err)
} }
// Record migration as applied // Record migration as applied
_, err = tx.Exec( _, err = tx.Exec(
"INSERT INTO schema_migrations (version, description, applied_at) VALUES (?, ?, ?)", "INSERT INTO schema_migrations (version, description, applied_at) VALUES (?, ?, ?)",
@ -172,16 +172,16 @@ func (m *MigrationManager) ApplyMigrations() error {
tx.Rollback() tx.Rollback()
return fmt.Errorf("failed to record migration %d: %w", migration.Version, err) return fmt.Errorf("failed to record migration %d: %w", migration.Version, err)
} }
// Commit transaction // Commit transaction
err = tx.Commit() err = tx.Commit()
if err != nil { if err != nil {
return fmt.Errorf("failed to commit migration %d: %w", migration.Version, err) return fmt.Errorf("failed to commit migration %d: %w", migration.Version, err)
} }
fmt.Printf("Successfully applied migration %d\n", migration.Version) fmt.Printf("Successfully applied migration %d\n", migration.Version)
} }
return nil return nil
} }
@ -203,7 +203,7 @@ func (m *MigrationManager) GetAppliedMigrations() ([]AppliedMigration, error) {
return nil, fmt.Errorf("failed to query applied migrations: %w", err) return nil, fmt.Errorf("failed to query applied migrations: %w", err)
} }
defer rows.Close() defer rows.Close()
var migrations []AppliedMigration var migrations []AppliedMigration
for rows.Next() { for rows.Next() {
var migration AppliedMigration var migration AppliedMigration
@ -213,7 +213,7 @@ func (m *MigrationManager) GetAppliedMigrations() ([]AppliedMigration, error) {
} }
migrations = append(migrations, migration) migrations = append(migrations, migration)
} }
return migrations, nil return migrations, nil
} }
@ -223,17 +223,17 @@ func (m *MigrationManager) ValidateSchema() error {
if err != nil { if err != nil {
return fmt.Errorf("failed to get current version: %w", err) return fmt.Errorf("failed to get current version: %w", err)
} }
migrations := GetMigrations() migrations := GetMigrations()
if len(migrations) == 0 { if len(migrations) == 0 {
return nil return nil
} }
latestVersion := migrations[len(migrations)-1].Version latestVersion := migrations[len(migrations)-1].Version
if currentVersion < latestVersion { if currentVersion < latestVersion {
return fmt.Errorf("schema is outdated: current version %d, latest version %d", currentVersion, latestVersion) return fmt.Errorf("schema is outdated: current version %d, latest version %d", currentVersion, latestVersion)
} }
return nil return nil
} }
@ -253,21 +253,21 @@ func getCurrentTimestamp() int64 {
func CreateDatabase(dbPath string) (*sql.DB, error) { func CreateDatabase(dbPath string) (*sql.DB, error) {
// TODO: Support different database types (PostgreSQL, MySQL, etc.) // TODO: Support different database types (PostgreSQL, MySQL, etc.)
// ASSUMPTION: Using SQLite for now, can be extended for other databases // ASSUMPTION: Using SQLite for now, can be extended for other databases
db, err := sql.Open("sqlite3", dbPath) db, err := sql.Open("sqlite3", dbPath)
if err != nil { if err != nil {
return nil, fmt.Errorf("failed to open database: %w", err) return nil, fmt.Errorf("failed to open database: %w", err)
} }
// Configure SQLite for better performance // Configure SQLite for better performance
pragmas := []string{ pragmas := []string{
"PRAGMA journal_mode=WAL", // Write-Ahead Logging for better concurrency
"PRAGMA synchronous=NORMAL", // Balance between safety and performance
"PRAGMA cache_size=10000", // Increase cache size
"PRAGMA foreign_keys=ON", // Enable foreign key constraints
"PRAGMA temp_store=MEMORY", // Store temporary tables in memory
"PRAGMA journal_mode=WAL", // Write-Ahead Logging for better concurrency
"PRAGMA synchronous=NORMAL", // Balance between safety and performance
"PRAGMA cache_size=10000", // Increase cache size
"PRAGMA foreign_keys=ON", // Enable foreign key constraints
"PRAGMA temp_store=MEMORY", // Store temporary tables in memory
} }
for _, pragma := range pragmas { for _, pragma := range pragmas {
_, err := db.Exec(pragma) _, err := db.Exec(pragma)
if err != nil { if err != nil {
@ -275,7 +275,7 @@ func CreateDatabase(dbPath string) (*sql.DB, error) {
return nil, fmt.Errorf("failed to set pragma %s: %w", pragma, err) return nil, fmt.Errorf("failed to set pragma %s: %w", pragma, err)
} }
} }
// Apply migrations // Apply migrations
migrationManager := NewMigrationManager(db) migrationManager := NewMigrationManager(db)
err = migrationManager.ApplyMigrations() err = migrationManager.ApplyMigrations()
@ -283,7 +283,7 @@ func CreateDatabase(dbPath string) (*sql.DB, error) {
db.Close() db.Close()
return nil, fmt.Errorf("failed to apply migrations: %w", err) return nil, fmt.Errorf("failed to apply migrations: %w", err)
} }
return db, nil return db, nil
} }

10
weed/mq/schema/flat_schema_utils_test.go

@ -248,11 +248,11 @@ func TestValidateKeyColumns(t *testing.T) {
// Helper function to check if string contains substring // Helper function to check if string contains substring
func contains(str, substr string) bool { func contains(str, substr string) bool {
return len(str) >= len(substr) &&
(len(substr) == 0 || str[len(str)-len(substr):] == substr ||
str[:len(substr)] == substr ||
len(str) > len(substr) && (str[len(str)-len(substr)-1:len(str)-len(substr)] == " " || str[len(str)-len(substr)-1] == ' ') && str[len(str)-len(substr):] == substr ||
findInString(str, substr))
return len(str) >= len(substr) &&
(len(substr) == 0 || str[len(str)-len(substr):] == substr ||
str[:len(substr)] == substr ||
len(str) > len(substr) && (str[len(str)-len(substr)-1:len(str)-len(substr)] == " " || str[len(str)-len(substr)-1] == ' ') && str[len(str)-len(substr):] == substr ||
findInString(str, substr))
} }
func findInString(str, substr string) bool { func findInString(str, substr string) bool {

2
weed/pb/mq_agent_pb/publish_response_test.go

@ -1,8 +1,8 @@
package mq_agent_pb package mq_agent_pb
import ( import (
"testing"
"google.golang.org/protobuf/proto" "google.golang.org/protobuf/proto"
"testing"
) )
func TestPublishRecordResponseSerialization(t *testing.T) { func TestPublishRecordResponseSerialization(t *testing.T) {

6
weed/pb/schema_pb/offset_test.go

@ -1,8 +1,8 @@
package schema_pb package schema_pb
import ( import (
"testing"
"google.golang.org/protobuf/proto" "google.golang.org/protobuf/proto"
"testing"
) )
func TestOffsetTypeEnums(t *testing.T) { func TestOffsetTypeEnums(t *testing.T) {
@ -34,8 +34,8 @@ func TestPartitionOffsetSerialization(t *testing.T) {
RangeStop: 31, RangeStop: 31,
UnixTimeNs: 1234567890, UnixTimeNs: 1234567890,
}, },
StartTsNs: 1234567890,
StartOffset: 42, // New field
StartTsNs: 1234567890,
StartOffset: 42, // New field
} }
// Test proto marshaling/unmarshaling // Test proto marshaling/unmarshaling

7
weed/pb/volume_server.proto

@ -525,6 +525,13 @@ message VolumeInfo {
int64 dat_file_size = 5; // store the original dat file size int64 dat_file_size = 5; // store the original dat file size
uint64 expire_at_sec = 6; // expiration time of ec volume uint64 expire_at_sec = 6; // expiration time of ec volume
bool read_only = 7; bool read_only = 7;
EcShardConfig ec_shard_config = 8; // EC shard configuration (optional, null = use default 10+4)
}
// EcShardConfig specifies erasure coding shard configuration
message EcShardConfig {
uint32 data_shards = 1; // Number of data shards (e.g., 10)
uint32 parity_shards = 2; // Number of parity shards (e.g., 4)
} }
message OldVersionVolumeInfo { message OldVersionVolumeInfo {
repeated RemoteFile files = 1; repeated RemoteFile files = 1;

502
weed/pb/volume_server_pb/volume_server.pb.go
File diff suppressed because it is too large
View File

18
weed/remote_storage/azure/azure_storage_client_test.go

@ -229,22 +229,22 @@ func TestToMetadata(t *testing.T) {
s3_constants.AmzUserMetaPrefix + "789": []byte("value3"), s3_constants.AmzUserMetaPrefix + "789": []byte("value3"),
}, },
expected: map[string]*string{ expected: map[string]*string{
"_123key": stringPtr("value1"), // starts with digit -> prefix _
"_456_2d_test": stringPtr("value2"), // starts with digit AND has dash
"_789": stringPtr("value3"),
"_123key": stringPtr("value1"), // starts with digit -> prefix _
"_456_2d_test": stringPtr("value2"), // starts with digit AND has dash
"_789": stringPtr("value3"),
}, },
}, },
{ {
name: "uppercase and mixed case keys", name: "uppercase and mixed case keys",
input: map[string][]byte{ input: map[string][]byte{
s3_constants.AmzUserMetaPrefix + "My-Key": []byte("value1"),
s3_constants.AmzUserMetaPrefix + "UPPERCASE": []byte("value2"),
s3_constants.AmzUserMetaPrefix + "MiXeD-CaSe": []byte("value3"),
s3_constants.AmzUserMetaPrefix + "My-Key": []byte("value1"),
s3_constants.AmzUserMetaPrefix + "UPPERCASE": []byte("value2"),
s3_constants.AmzUserMetaPrefix + "MiXeD-CaSe": []byte("value3"),
}, },
expected: map[string]*string{ expected: map[string]*string{
"my_2d_key": stringPtr("value1"), // lowercase + dash -> _2d_
"uppercase": stringPtr("value2"),
"mixed_2d_case": stringPtr("value3"),
"my_2d_key": stringPtr("value1"), // lowercase + dash -> _2d_
"uppercase": stringPtr("value2"),
"mixed_2d_case": stringPtr("value3"),
}, },
}, },
{ {

2
weed/s3api/s3_sse_s3_integration_test.go

@ -78,7 +78,7 @@ func TestSSES3EndToEndSmallFile(t *testing.T) {
// Step 3: Decrypt (simulates what happens during GET) // Step 3: Decrypt (simulates what happens during GET)
// This tests the IV retrieval path for inline files // This tests the IV retrieval path for inline files
// First, deserialize metadata from storage // First, deserialize metadata from storage
retrievedKeyData := mockEntry.Extended[s3_constants.SeaweedFSSSES3Key] retrievedKeyData := mockEntry.Extended[s3_constants.SeaweedFSSSES3Key]
retrievedKey, err := DeserializeSSES3Metadata(retrievedKeyData, keyManager) retrievedKey, err := DeserializeSSES3Metadata(retrievedKeyData, keyManager)

10
weed/s3api/s3_validation_utils.go

@ -71,7 +71,7 @@ func ValidateSSES3Key(sseKey *SSES3Key) error {
if sseKey == nil { if sseKey == nil {
return fmt.Errorf("SSE-S3 key cannot be nil") return fmt.Errorf("SSE-S3 key cannot be nil")
} }
// Validate key bytes // Validate key bytes
if sseKey.Key == nil { if sseKey.Key == nil {
return fmt.Errorf("SSE-S3 key bytes cannot be nil") return fmt.Errorf("SSE-S3 key bytes cannot be nil")
@ -79,22 +79,22 @@ func ValidateSSES3Key(sseKey *SSES3Key) error {
if len(sseKey.Key) != SSES3KeySize { if len(sseKey.Key) != SSES3KeySize {
return fmt.Errorf("invalid SSE-S3 key size: expected %d bytes, got %d", SSES3KeySize, len(sseKey.Key)) return fmt.Errorf("invalid SSE-S3 key size: expected %d bytes, got %d", SSES3KeySize, len(sseKey.Key))
} }
// Validate algorithm // Validate algorithm
if sseKey.Algorithm != SSES3Algorithm { if sseKey.Algorithm != SSES3Algorithm {
return fmt.Errorf("invalid SSE-S3 algorithm: expected %q, got %q", SSES3Algorithm, sseKey.Algorithm) return fmt.Errorf("invalid SSE-S3 algorithm: expected %q, got %q", SSES3Algorithm, sseKey.Algorithm)
} }
// Validate key ID (should not be empty) // Validate key ID (should not be empty)
if sseKey.KeyID == "" { if sseKey.KeyID == "" {
return fmt.Errorf("SSE-S3 key ID cannot be empty") return fmt.Errorf("SSE-S3 key ID cannot be empty")
} }
// IV validation is optional during key creation - it will be set during encryption // IV validation is optional during key creation - it will be set during encryption
// If IV is set, validate its length // If IV is set, validate its length
if len(sseKey.IV) > 0 && len(sseKey.IV) != s3_constants.AESBlockSize { if len(sseKey.IV) > 0 && len(sseKey.IV) != s3_constants.AESBlockSize {
return fmt.Errorf("invalid SSE-S3 IV length: expected %d bytes, got %d", s3_constants.AESBlockSize, len(sseKey.IV)) return fmt.Errorf("invalid SSE-S3 IV length: expected %d bytes, got %d", s3_constants.AESBlockSize, len(sseKey.IV))
} }
return nil return nil
} }

64
weed/server/volume_grpc_erasure_coding.go

@ -50,20 +50,38 @@ func (vs *VolumeServer) VolumeEcShardsGenerate(ctx context.Context, req *volume_
return nil, fmt.Errorf("existing collection:%v unexpected input: %v", v.Collection, req.Collection) return nil, fmt.Errorf("existing collection:%v unexpected input: %v", v.Collection, req.Collection)
} }
// Create EC context - prefer existing .vif config if present (for regeneration scenarios)
ecCtx := erasure_coding.NewDefaultECContext(req.Collection, needle.VolumeId(req.VolumeId))
if volumeInfo, _, found, _ := volume_info.MaybeLoadVolumeInfo(baseFileName + ".vif"); found && volumeInfo.EcShardConfig != nil {
ds := int(volumeInfo.EcShardConfig.DataShards)
ps := int(volumeInfo.EcShardConfig.ParityShards)
// Validate and use existing EC config
if ds > 0 && ps > 0 && ds+ps <= erasure_coding.MaxShardCount {
ecCtx.DataShards = ds
ecCtx.ParityShards = ps
glog.V(0).Infof("Using existing EC config for volume %d: %s", req.VolumeId, ecCtx.String())
} else {
glog.Warningf("Invalid EC config in .vif for volume %d (data=%d, parity=%d), using defaults", req.VolumeId, ds, ps)
}
} else {
glog.V(0).Infof("Using default EC config for volume %d: %s", req.VolumeId, ecCtx.String())
}
shouldCleanup := true shouldCleanup := true
defer func() { defer func() {
if !shouldCleanup { if !shouldCleanup {
return return
} }
for i := 0; i < erasure_coding.TotalShardsCount; i++ {
os.Remove(fmt.Sprintf("%s.ec%2d", baseFileName, i))
for i := 0; i < ecCtx.Total(); i++ {
os.Remove(baseFileName + ecCtx.ToExt(i))
} }
os.Remove(v.IndexFileName() + ".ecx") os.Remove(v.IndexFileName() + ".ecx")
}() }()
// write .ec00 ~ .ec13 files
if err := erasure_coding.WriteEcFiles(baseFileName); err != nil {
return nil, fmt.Errorf("WriteEcFiles %s: %v", baseFileName, err)
// write .ec00 ~ .ec[TotalShards-1] files using context
if err := erasure_coding.WriteEcFilesWithContext(baseFileName, ecCtx); err != nil {
return nil, fmt.Errorf("WriteEcFilesWithContext %s: %v", baseFileName, err)
} }
// write .ecx file // write .ecx file
@ -84,6 +102,21 @@ func (vs *VolumeServer) VolumeEcShardsGenerate(ctx context.Context, req *volume_
datSize, _, _ := v.FileStat() datSize, _, _ := v.FileStat()
volumeInfo.DatFileSize = int64(datSize) volumeInfo.DatFileSize = int64(datSize)
// Validate EC configuration before saving to .vif
if ecCtx.DataShards <= 0 || ecCtx.ParityShards <= 0 || ecCtx.Total() > erasure_coding.MaxShardCount {
return nil, fmt.Errorf("invalid EC config before saving: data=%d, parity=%d, total=%d (max=%d)",
ecCtx.DataShards, ecCtx.ParityShards, ecCtx.Total(), erasure_coding.MaxShardCount)
}
// Save EC configuration to VolumeInfo
volumeInfo.EcShardConfig = &volume_server_pb.EcShardConfig{
DataShards: uint32(ecCtx.DataShards),
ParityShards: uint32(ecCtx.ParityShards),
}
glog.V(1).Infof("Saving EC config to .vif for volume %d: %d+%d (total: %d)",
req.VolumeId, ecCtx.DataShards, ecCtx.ParityShards, ecCtx.Total())
if err := volume_info.SaveVolumeInfo(baseFileName+".vif", volumeInfo); err != nil { if err := volume_info.SaveVolumeInfo(baseFileName+".vif", volumeInfo); err != nil {
return nil, fmt.Errorf("SaveVolumeInfo %s: %v", baseFileName, err) return nil, fmt.Errorf("SaveVolumeInfo %s: %v", baseFileName, err)
} }
@ -442,9 +475,10 @@ func (vs *VolumeServer) VolumeEcShardsToVolume(ctx context.Context, req *volume_
glog.V(0).Infof("VolumeEcShardsToVolume: %v", req) glog.V(0).Infof("VolumeEcShardsToVolume: %v", req)
// collect .ec00 ~ .ec09 files
shardFileNames := make([]string, erasure_coding.DataShardsCount)
v, found := vs.store.CollectEcShards(needle.VolumeId(req.VolumeId), shardFileNames)
// Collect all EC shards (NewEcVolume will load EC config from .vif into v.ECContext)
// Use MaxShardCount (32) to support custom EC ratios up to 32 total shards
tempShards := make([]string, erasure_coding.MaxShardCount)
v, found := vs.store.CollectEcShards(needle.VolumeId(req.VolumeId), tempShards)
if !found { if !found {
return nil, fmt.Errorf("ec volume %d not found", req.VolumeId) return nil, fmt.Errorf("ec volume %d not found", req.VolumeId)
} }
@ -453,7 +487,19 @@ func (vs *VolumeServer) VolumeEcShardsToVolume(ctx context.Context, req *volume_
return nil, fmt.Errorf("existing collection:%v unexpected input: %v", v.Collection, req.Collection) return nil, fmt.Errorf("existing collection:%v unexpected input: %v", v.Collection, req.Collection)
} }
for shardId := 0; shardId < erasure_coding.DataShardsCount; shardId++ {
// Use EC context (already loaded from .vif) to determine data shard count
dataShards := v.ECContext.DataShards
// Defensive validation to prevent panics from corrupted ECContext
if dataShards <= 0 || dataShards > erasure_coding.MaxShardCount {
return nil, fmt.Errorf("invalid data shard count %d for volume %d (must be 1..%d)", dataShards, req.VolumeId, erasure_coding.MaxShardCount)
}
shardFileNames := tempShards[:dataShards]
glog.V(1).Infof("Using EC config from volume %d: %d data shards", req.VolumeId, dataShards)
// Verify all data shards are present
for shardId := 0; shardId < dataShards; shardId++ {
if shardFileNames[shardId] == "" { if shardFileNames[shardId] == "" {
return nil, fmt.Errorf("ec volume %d missing shard %d", req.VolumeId, shardId) return nil, fmt.Errorf("ec volume %d missing shard %d", req.VolumeId, shardId)
} }

14
weed/shell/command_ec_common.go

@ -622,7 +622,8 @@ func (ecb *ecBalancer) deleteDuplicatedEcShards(collection string) error {
func (ecb *ecBalancer) doDeduplicateEcShards(collection string, vid needle.VolumeId, locations []*EcNode) error { func (ecb *ecBalancer) doDeduplicateEcShards(collection string, vid needle.VolumeId, locations []*EcNode) error {
// check whether this volume has ecNodes that are over average // check whether this volume has ecNodes that are over average
shardToLocations := make([][]*EcNode, erasure_coding.TotalShardsCount)
// Use MaxShardCount (32) to support custom EC ratios
shardToLocations := make([][]*EcNode, erasure_coding.MaxShardCount)
for _, ecNode := range locations { for _, ecNode := range locations {
shardBits := findEcVolumeShards(ecNode, vid) shardBits := findEcVolumeShards(ecNode, vid)
for _, shardId := range shardBits.ShardIds() { for _, shardId := range shardBits.ShardIds() {
@ -677,11 +678,16 @@ func countShardsByRack(vid needle.VolumeId, locations []*EcNode) map[string]int
func (ecb *ecBalancer) doBalanceEcShardsAcrossRacks(collection string, vid needle.VolumeId, locations []*EcNode) error { func (ecb *ecBalancer) doBalanceEcShardsAcrossRacks(collection string, vid needle.VolumeId, locations []*EcNode) error {
racks := ecb.racks() racks := ecb.racks()
// calculate average number of shards an ec rack should have for one volume
averageShardsPerEcRack := ceilDivide(erasure_coding.TotalShardsCount, len(racks))
// see the volume's shards are in how many racks, and how many in each rack // see the volume's shards are in how many racks, and how many in each rack
rackToShardCount := countShardsByRack(vid, locations) rackToShardCount := countShardsByRack(vid, locations)
// Calculate actual total shards for this volume (not hardcoded default)
var totalShardsForVolume int
for _, count := range rackToShardCount {
totalShardsForVolume += count
}
// calculate average number of shards an ec rack should have for one volume
averageShardsPerEcRack := ceilDivide(totalShardsForVolume, len(racks))
rackEcNodesWithVid := groupBy(locations, func(ecNode *EcNode) string { rackEcNodesWithVid := groupBy(locations, func(ecNode *EcNode) string {
return string(ecNode.rack) return string(ecNode.rack)
}) })

3
weed/shell/command_ec_rebuild.go

@ -264,7 +264,8 @@ func (ecShardMap EcShardMap) registerEcNode(ecNode *EcNode, collection string) {
if shardInfo.Collection == collection { if shardInfo.Collection == collection {
existing, found := ecShardMap[needle.VolumeId(shardInfo.Id)] existing, found := ecShardMap[needle.VolumeId(shardInfo.Id)]
if !found { if !found {
existing = make([][]*EcNode, erasure_coding.TotalShardsCount)
// Use MaxShardCount (32) to support custom EC ratios
existing = make([][]*EcNode, erasure_coding.MaxShardCount)
ecShardMap[needle.VolumeId(shardInfo.Id)] = existing ecShardMap[needle.VolumeId(shardInfo.Id)] = existing
} }
for _, shardId := range erasure_coding.ShardBits(shardInfo.EcIndexBits).ShardIds() { for _, shardId := range erasure_coding.ShardBits(shardInfo.EcIndexBits).ShardIds() {

13
weed/storage/disk_location_ec.go

@ -16,7 +16,9 @@ import (
) )
var ( var (
re = regexp.MustCompile(`\.ec[0-9][0-9]`)
// Match .ec00 through .ec999 (currently only .ec00-.ec31 are used)
// Using \d{2,3} for future-proofing if MaxShardCount is ever increased beyond 99
re = regexp.MustCompile(`\.ec\d{2,3}`)
) )
func (l *DiskLocation) FindEcVolume(vid needle.VolumeId) (*erasure_coding.EcVolume, bool) { func (l *DiskLocation) FindEcVolume(vid needle.VolumeId) (*erasure_coding.EcVolume, bool) {
@ -398,8 +400,8 @@ func (l *DiskLocation) validateEcVolume(collection string, vid needle.VolumeId)
var actualShardSize int64 = -1 var actualShardSize int64 = -1
// Count shards and validate they all have the same size (required for Reed-Solomon EC) // Count shards and validate they all have the same size (required for Reed-Solomon EC)
// Shard files (.ec00 - .ec13) are always in l.Directory, not l.IdxDirectory
for i := 0; i < erasure_coding.TotalShardsCount; i++ {
// Check up to MaxShardCount (32) to support custom EC ratios
for i := 0; i < erasure_coding.MaxShardCount; i++ {
shardFileName := baseFileName + erasure_coding.ToExt(i) shardFileName := baseFileName + erasure_coding.ToExt(i)
fi, err := os.Stat(shardFileName) fi, err := os.Stat(shardFileName)
@ -472,8 +474,9 @@ func (l *DiskLocation) removeEcVolumeFiles(collection string, vid needle.VolumeI
removeFile(indexBaseFileName+".ecx", "EC index file") removeFile(indexBaseFileName+".ecx", "EC index file")
removeFile(indexBaseFileName+".ecj", "EC journal file") removeFile(indexBaseFileName+".ecj", "EC journal file")
// Remove all EC shard files (.ec00 ~ .ec13) from data directory
for i := 0; i < erasure_coding.TotalShardsCount; i++ {
// Remove all EC shard files (.ec00 ~ .ec31) from data directory
// Use MaxShardCount (32) to support custom EC ratios
for i := 0; i < erasure_coding.MaxShardCount; i++ {
removeFile(baseFileName+erasure_coding.ToExt(i), "EC shard file") removeFile(baseFileName+erasure_coding.ToExt(i), "EC shard file")
} }
} }

46
weed/storage/erasure_coding/ec_context.go

@ -0,0 +1,46 @@
package erasure_coding
import (
"fmt"
"github.com/klauspost/reedsolomon"
"github.com/seaweedfs/seaweedfs/weed/storage/needle"
)
// ECContext encapsulates erasure coding parameters for encoding/decoding operations
type ECContext struct {
DataShards int
ParityShards int
Collection string
VolumeId needle.VolumeId
}
// Total returns the total number of shards (data + parity)
func (ctx *ECContext) Total() int {
return ctx.DataShards + ctx.ParityShards
}
// NewDefaultECContext creates a context with default 10+4 shard configuration
func NewDefaultECContext(collection string, volumeId needle.VolumeId) *ECContext {
return &ECContext{
DataShards: DataShardsCount,
ParityShards: ParityShardsCount,
Collection: collection,
VolumeId: volumeId,
}
}
// CreateEncoder creates a Reed-Solomon encoder for this context
func (ctx *ECContext) CreateEncoder() (reedsolomon.Encoder, error) {
return reedsolomon.New(ctx.DataShards, ctx.ParityShards)
}
// ToExt returns the file extension for a given shard index
func (ctx *ECContext) ToExt(shardIndex int) string {
return fmt.Sprintf(".ec%02d", shardIndex)
}
// String returns a human-readable representation of the EC configuration
func (ctx *ECContext) String() string {
return fmt.Sprintf("%d+%d (total: %d)", ctx.DataShards, ctx.ParityShards, ctx.Total())
}

113
weed/storage/erasure_coding/ec_encoder.go

@ -11,6 +11,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/storage/idx" "github.com/seaweedfs/seaweedfs/weed/storage/idx"
"github.com/seaweedfs/seaweedfs/weed/storage/needle_map" "github.com/seaweedfs/seaweedfs/weed/storage/needle_map"
"github.com/seaweedfs/seaweedfs/weed/storage/types" "github.com/seaweedfs/seaweedfs/weed/storage/types"
"github.com/seaweedfs/seaweedfs/weed/storage/volume_info"
"github.com/seaweedfs/seaweedfs/weed/util" "github.com/seaweedfs/seaweedfs/weed/util"
) )
@ -18,6 +19,7 @@ const (
DataShardsCount = 10 DataShardsCount = 10
ParityShardsCount = 4 ParityShardsCount = 4
TotalShardsCount = DataShardsCount + ParityShardsCount TotalShardsCount = DataShardsCount + ParityShardsCount
MaxShardCount = 32 // Maximum number of shards since ShardBits is uint32 (bits 0-31)
MinTotalDisks = TotalShardsCount/ParityShardsCount + 1 MinTotalDisks = TotalShardsCount/ParityShardsCount + 1
ErasureCodingLargeBlockSize = 1024 * 1024 * 1024 // 1GB ErasureCodingLargeBlockSize = 1024 * 1024 * 1024 // 1GB
ErasureCodingSmallBlockSize = 1024 * 1024 // 1MB ErasureCodingSmallBlockSize = 1024 * 1024 // 1MB
@ -54,20 +56,53 @@ func WriteSortedFileFromIdx(baseFileName string, ext string) (e error) {
return nil return nil
} }
// WriteEcFiles generates .ec00 ~ .ec13 files
// WriteEcFiles generates .ec00 ~ .ec13 files using default EC context
func WriteEcFiles(baseFileName string) error { func WriteEcFiles(baseFileName string) error {
return generateEcFiles(baseFileName, 256*1024, ErasureCodingLargeBlockSize, ErasureCodingSmallBlockSize)
ctx := NewDefaultECContext("", 0)
return WriteEcFilesWithContext(baseFileName, ctx)
}
// WriteEcFilesWithContext generates EC files using the provided context
func WriteEcFilesWithContext(baseFileName string, ctx *ECContext) error {
return generateEcFiles(baseFileName, 256*1024, ErasureCodingLargeBlockSize, ErasureCodingSmallBlockSize, ctx)
} }
func RebuildEcFiles(baseFileName string) ([]uint32, error) { func RebuildEcFiles(baseFileName string) ([]uint32, error) {
return generateMissingEcFiles(baseFileName, 256*1024, ErasureCodingLargeBlockSize, ErasureCodingSmallBlockSize)
// Attempt to load EC config from .vif file to preserve original configuration
var ctx *ECContext
if volumeInfo, _, found, _ := volume_info.MaybeLoadVolumeInfo(baseFileName + ".vif"); found && volumeInfo.EcShardConfig != nil {
ds := int(volumeInfo.EcShardConfig.DataShards)
ps := int(volumeInfo.EcShardConfig.ParityShards)
// Validate EC config before using it
if ds > 0 && ps > 0 && ds+ps <= MaxShardCount {
ctx = &ECContext{
DataShards: ds,
ParityShards: ps,
}
glog.V(0).Infof("Rebuilding EC files for %s with config from .vif: %s", baseFileName, ctx.String())
} else {
glog.Warningf("Invalid EC config in .vif for %s (data=%d, parity=%d), using default", baseFileName, ds, ps)
ctx = NewDefaultECContext("", 0)
}
} else {
glog.V(0).Infof("Rebuilding EC files for %s with default config", baseFileName)
ctx = NewDefaultECContext("", 0)
}
return RebuildEcFilesWithContext(baseFileName, ctx)
}
// RebuildEcFilesWithContext rebuilds missing EC files using the provided context
func RebuildEcFilesWithContext(baseFileName string, ctx *ECContext) ([]uint32, error) {
return generateMissingEcFiles(baseFileName, 256*1024, ErasureCodingLargeBlockSize, ErasureCodingSmallBlockSize, ctx)
} }
func ToExt(ecIndex int) string { func ToExt(ecIndex int) string {
return fmt.Sprintf(".ec%02d", ecIndex) return fmt.Sprintf(".ec%02d", ecIndex)
} }
func generateEcFiles(baseFileName string, bufferSize int, largeBlockSize int64, smallBlockSize int64) error {
func generateEcFiles(baseFileName string, bufferSize int, largeBlockSize int64, smallBlockSize int64, ctx *ECContext) error {
file, err := os.OpenFile(baseFileName+".dat", os.O_RDONLY, 0) file, err := os.OpenFile(baseFileName+".dat", os.O_RDONLY, 0)
if err != nil { if err != nil {
return fmt.Errorf("failed to open dat file: %w", err) return fmt.Errorf("failed to open dat file: %w", err)
@ -79,21 +114,21 @@ func generateEcFiles(baseFileName string, bufferSize int, largeBlockSize int64,
return fmt.Errorf("failed to stat dat file: %w", err) return fmt.Errorf("failed to stat dat file: %w", err)
} }
glog.V(0).Infof("encodeDatFile %s.dat size:%d", baseFileName, fi.Size())
err = encodeDatFile(fi.Size(), baseFileName, bufferSize, largeBlockSize, file, smallBlockSize)
glog.V(0).Infof("encodeDatFile %s.dat size:%d with EC context %s", baseFileName, fi.Size(), ctx.String())
err = encodeDatFile(fi.Size(), baseFileName, bufferSize, largeBlockSize, file, smallBlockSize, ctx)
if err != nil { if err != nil {
return fmt.Errorf("encodeDatFile: %w", err) return fmt.Errorf("encodeDatFile: %w", err)
} }
return nil return nil
} }
func generateMissingEcFiles(baseFileName string, bufferSize int, largeBlockSize int64, smallBlockSize int64) (generatedShardIds []uint32, err error) {
func generateMissingEcFiles(baseFileName string, bufferSize int, largeBlockSize int64, smallBlockSize int64, ctx *ECContext) (generatedShardIds []uint32, err error) {
shardHasData := make([]bool, TotalShardsCount)
inputFiles := make([]*os.File, TotalShardsCount)
outputFiles := make([]*os.File, TotalShardsCount)
for shardId := 0; shardId < TotalShardsCount; shardId++ {
shardFileName := baseFileName + ToExt(shardId)
shardHasData := make([]bool, ctx.Total())
inputFiles := make([]*os.File, ctx.Total())
outputFiles := make([]*os.File, ctx.Total())
for shardId := 0; shardId < ctx.Total(); shardId++ {
shardFileName := baseFileName + ctx.ToExt(shardId)
if util.FileExists(shardFileName) { if util.FileExists(shardFileName) {
shardHasData[shardId] = true shardHasData[shardId] = true
inputFiles[shardId], err = os.OpenFile(shardFileName, os.O_RDONLY, 0) inputFiles[shardId], err = os.OpenFile(shardFileName, os.O_RDONLY, 0)
@ -111,14 +146,14 @@ func generateMissingEcFiles(baseFileName string, bufferSize int, largeBlockSize
} }
} }
err = rebuildEcFiles(shardHasData, inputFiles, outputFiles)
err = rebuildEcFiles(shardHasData, inputFiles, outputFiles, ctx)
if err != nil { if err != nil {
return nil, fmt.Errorf("rebuildEcFiles: %w", err) return nil, fmt.Errorf("rebuildEcFiles: %w", err)
} }
return return
} }
func encodeData(file *os.File, enc reedsolomon.Encoder, startOffset, blockSize int64, buffers [][]byte, outputs []*os.File) error {
func encodeData(file *os.File, enc reedsolomon.Encoder, startOffset, blockSize int64, buffers [][]byte, outputs []*os.File, ctx *ECContext) error {
bufferSize := int64(len(buffers[0])) bufferSize := int64(len(buffers[0]))
if bufferSize == 0 { if bufferSize == 0 {
@ -131,7 +166,7 @@ func encodeData(file *os.File, enc reedsolomon.Encoder, startOffset, blockSize i
} }
for b := int64(0); b < batchCount; b++ { for b := int64(0); b < batchCount; b++ {
err := encodeDataOneBatch(file, enc, startOffset+b*bufferSize, blockSize, buffers, outputs)
err := encodeDataOneBatch(file, enc, startOffset+b*bufferSize, blockSize, buffers, outputs, ctx)
if err != nil { if err != nil {
return err return err
} }
@ -140,9 +175,9 @@ func encodeData(file *os.File, enc reedsolomon.Encoder, startOffset, blockSize i
return nil return nil
} }
func openEcFiles(baseFileName string, forRead bool) (files []*os.File, err error) {
for i := 0; i < TotalShardsCount; i++ {
fname := baseFileName + ToExt(i)
func openEcFiles(baseFileName string, forRead bool, ctx *ECContext) (files []*os.File, err error) {
for i := 0; i < ctx.Total(); i++ {
fname := baseFileName + ctx.ToExt(i)
openOption := os.O_TRUNC | os.O_CREATE | os.O_WRONLY openOption := os.O_TRUNC | os.O_CREATE | os.O_WRONLY
if forRead { if forRead {
openOption = os.O_RDONLY openOption = os.O_RDONLY
@ -164,10 +199,10 @@ func closeEcFiles(files []*os.File) {
} }
} }
func encodeDataOneBatch(file *os.File, enc reedsolomon.Encoder, startOffset, blockSize int64, buffers [][]byte, outputs []*os.File) error {
func encodeDataOneBatch(file *os.File, enc reedsolomon.Encoder, startOffset, blockSize int64, buffers [][]byte, outputs []*os.File, ctx *ECContext) error {
// read data into buffers // read data into buffers
for i := 0; i < DataShardsCount; i++ {
for i := 0; i < ctx.DataShards; i++ {
n, err := file.ReadAt(buffers[i], startOffset+blockSize*int64(i)) n, err := file.ReadAt(buffers[i], startOffset+blockSize*int64(i))
if err != nil { if err != nil {
if err != io.EOF { if err != io.EOF {
@ -186,7 +221,7 @@ func encodeDataOneBatch(file *os.File, enc reedsolomon.Encoder, startOffset, blo
return err return err
} }
for i := 0; i < TotalShardsCount; i++ {
for i := 0; i < ctx.Total(); i++ {
_, err := outputs[i].Write(buffers[i]) _, err := outputs[i].Write(buffers[i])
if err != nil { if err != nil {
return err return err
@ -196,53 +231,57 @@ func encodeDataOneBatch(file *os.File, enc reedsolomon.Encoder, startOffset, blo
return nil return nil
} }
func encodeDatFile(remainingSize int64, baseFileName string, bufferSize int, largeBlockSize int64, file *os.File, smallBlockSize int64) error {
func encodeDatFile(remainingSize int64, baseFileName string, bufferSize int, largeBlockSize int64, file *os.File, smallBlockSize int64, ctx *ECContext) error {
var processedSize int64 var processedSize int64
enc, err := reedsolomon.New(DataShardsCount, ParityShardsCount)
enc, err := ctx.CreateEncoder()
if err != nil { if err != nil {
return fmt.Errorf("failed to create encoder: %w", err) return fmt.Errorf("failed to create encoder: %w", err)
} }
buffers := make([][]byte, TotalShardsCount)
buffers := make([][]byte, ctx.Total())
for i := range buffers { for i := range buffers {
buffers[i] = make([]byte, bufferSize) buffers[i] = make([]byte, bufferSize)
} }
outputs, err := openEcFiles(baseFileName, false)
outputs, err := openEcFiles(baseFileName, false, ctx)
defer closeEcFiles(outputs) defer closeEcFiles(outputs)
if err != nil { if err != nil {
return fmt.Errorf("failed to open ec files %s: %v", baseFileName, err) return fmt.Errorf("failed to open ec files %s: %v", baseFileName, err)
} }
for remainingSize > largeBlockSize*DataShardsCount {
err = encodeData(file, enc, processedSize, largeBlockSize, buffers, outputs)
// Pre-calculate row sizes to avoid redundant calculations in loops
largeRowSize := largeBlockSize * int64(ctx.DataShards)
smallRowSize := smallBlockSize * int64(ctx.DataShards)
for remainingSize >= largeRowSize {
err = encodeData(file, enc, processedSize, largeBlockSize, buffers, outputs, ctx)
if err != nil { if err != nil {
return fmt.Errorf("failed to encode large chunk data: %w", err) return fmt.Errorf("failed to encode large chunk data: %w", err)
} }
remainingSize -= largeBlockSize * DataShardsCount
processedSize += largeBlockSize * DataShardsCount
remainingSize -= largeRowSize
processedSize += largeRowSize
} }
for remainingSize > 0 { for remainingSize > 0 {
err = encodeData(file, enc, processedSize, smallBlockSize, buffers, outputs)
err = encodeData(file, enc, processedSize, smallBlockSize, buffers, outputs, ctx)
if err != nil { if err != nil {
return fmt.Errorf("failed to encode small chunk data: %w", err) return fmt.Errorf("failed to encode small chunk data: %w", err)
} }
remainingSize -= smallBlockSize * DataShardsCount
processedSize += smallBlockSize * DataShardsCount
remainingSize -= smallRowSize
processedSize += smallRowSize
} }
return nil return nil
} }
func rebuildEcFiles(shardHasData []bool, inputFiles []*os.File, outputFiles []*os.File) error {
func rebuildEcFiles(shardHasData []bool, inputFiles []*os.File, outputFiles []*os.File, ctx *ECContext) error {
enc, err := reedsolomon.New(DataShardsCount, ParityShardsCount)
enc, err := ctx.CreateEncoder()
if err != nil { if err != nil {
return fmt.Errorf("failed to create encoder: %w", err) return fmt.Errorf("failed to create encoder: %w", err)
} }
buffers := make([][]byte, TotalShardsCount)
buffers := make([][]byte, ctx.Total())
for i := range buffers { for i := range buffers {
if shardHasData[i] { if shardHasData[i] {
buffers[i] = make([]byte, ErasureCodingSmallBlockSize) buffers[i] = make([]byte, ErasureCodingSmallBlockSize)
@ -254,7 +293,7 @@ func rebuildEcFiles(shardHasData []bool, inputFiles []*os.File, outputFiles []*o
for { for {
// read the input data from files // read the input data from files
for i := 0; i < TotalShardsCount; i++ {
for i := 0; i < ctx.Total(); i++ {
if shardHasData[i] { if shardHasData[i] {
n, _ := inputFiles[i].ReadAt(buffers[i], startOffset) n, _ := inputFiles[i].ReadAt(buffers[i], startOffset)
if n == 0 { if n == 0 {
@ -278,7 +317,7 @@ func rebuildEcFiles(shardHasData []bool, inputFiles []*os.File, outputFiles []*o
} }
// write the data to output files // write the data to output files
for i := 0; i < TotalShardsCount; i++ {
for i := 0; i < ctx.Total(); i++ {
if !shardHasData[i] { if !shardHasData[i] {
n, _ := outputFiles[i].WriteAt(buffers[i][:inputBufferDataSize], startOffset) n, _ := outputFiles[i].WriteAt(buffers[i][:inputBufferDataSize], startOffset)
if inputBufferDataSize != n { if inputBufferDataSize != n {

19
weed/storage/erasure_coding/ec_test.go

@ -23,7 +23,10 @@ func TestEncodingDecoding(t *testing.T) {
bufferSize := 50 bufferSize := 50
baseFileName := "1" baseFileName := "1"
err := generateEcFiles(baseFileName, bufferSize, largeBlockSize, smallBlockSize)
// Create default EC context for testing
ctx := NewDefaultECContext("", 0)
err := generateEcFiles(baseFileName, bufferSize, largeBlockSize, smallBlockSize, ctx)
if err != nil { if err != nil {
t.Logf("generateEcFiles: %v", err) t.Logf("generateEcFiles: %v", err)
} }
@ -33,16 +36,16 @@ func TestEncodingDecoding(t *testing.T) {
t.Logf("WriteSortedFileFromIdx: %v", err) t.Logf("WriteSortedFileFromIdx: %v", err)
} }
err = validateFiles(baseFileName)
err = validateFiles(baseFileName, ctx)
if err != nil { if err != nil {
t.Logf("WriteSortedFileFromIdx: %v", err) t.Logf("WriteSortedFileFromIdx: %v", err)
} }
removeGeneratedFiles(baseFileName)
removeGeneratedFiles(baseFileName, ctx)
} }
func validateFiles(baseFileName string) error {
func validateFiles(baseFileName string, ctx *ECContext) error {
nm, err := readNeedleMap(baseFileName) nm, err := readNeedleMap(baseFileName)
if err != nil { if err != nil {
return fmt.Errorf("readNeedleMap: %v", err) return fmt.Errorf("readNeedleMap: %v", err)
@ -60,7 +63,7 @@ func validateFiles(baseFileName string) error {
return fmt.Errorf("failed to stat dat file: %v", err) return fmt.Errorf("failed to stat dat file: %v", err)
} }
ecFiles, err := openEcFiles(baseFileName, true)
ecFiles, err := openEcFiles(baseFileName, true, ctx)
if err != nil { if err != nil {
return fmt.Errorf("error opening ec files: %w", err) return fmt.Errorf("error opening ec files: %w", err)
} }
@ -184,9 +187,9 @@ func readFromFile(file *os.File, data []byte, ecFileOffset int64) (err error) {
return return
} }
func removeGeneratedFiles(baseFileName string) {
for i := 0; i < DataShardsCount+ParityShardsCount; i++ {
fname := fmt.Sprintf("%s.ec%02d", baseFileName, i)
func removeGeneratedFiles(baseFileName string, ctx *ECContext) {
for i := 0; i < ctx.Total(); i++ {
fname := baseFileName + ctx.ToExt(i)
os.Remove(fname) os.Remove(fname)
} }
os.Remove(baseFileName + ".ecx") os.Remove(baseFileName + ".ecx")

28
weed/storage/erasure_coding/ec_volume.go

@ -41,7 +41,8 @@ type EcVolume struct {
ecjFileAccessLock sync.Mutex ecjFileAccessLock sync.Mutex
diskType types.DiskType diskType types.DiskType
datFileSize int64 datFileSize int64
ExpireAtSec uint64 //ec volume destroy time, calculated from the ec volume was created
ExpireAtSec uint64 //ec volume destroy time, calculated from the ec volume was created
ECContext *ECContext // EC encoding parameters
} }
func NewEcVolume(diskType types.DiskType, dir string, dirIdx string, collection string, vid needle.VolumeId) (ev *EcVolume, err error) { func NewEcVolume(diskType types.DiskType, dir string, dirIdx string, collection string, vid needle.VolumeId) (ev *EcVolume, err error) {
@ -73,9 +74,32 @@ func NewEcVolume(diskType types.DiskType, dir string, dirIdx string, collection
ev.Version = needle.Version(volumeInfo.Version) ev.Version = needle.Version(volumeInfo.Version)
ev.datFileSize = volumeInfo.DatFileSize ev.datFileSize = volumeInfo.DatFileSize
ev.ExpireAtSec = volumeInfo.ExpireAtSec ev.ExpireAtSec = volumeInfo.ExpireAtSec
// Initialize EC context from .vif if present; fallback to defaults
if volumeInfo.EcShardConfig != nil {
ds := int(volumeInfo.EcShardConfig.DataShards)
ps := int(volumeInfo.EcShardConfig.ParityShards)
// Validate shard counts to prevent zero or invalid values
if ds <= 0 || ps <= 0 || ds+ps > MaxShardCount {
glog.Warningf("Invalid EC config in VolumeInfo for volume %d (data=%d, parity=%d), using defaults", vid, ds, ps)
ev.ECContext = NewDefaultECContext(collection, vid)
} else {
ev.ECContext = &ECContext{
Collection: collection,
VolumeId: vid,
DataShards: ds,
ParityShards: ps,
}
glog.V(1).Infof("Loaded EC config from VolumeInfo for volume %d: %s", vid, ev.ECContext.String())
}
} else {
ev.ECContext = NewDefaultECContext(collection, vid)
}
} else { } else {
glog.Warningf("vif file not found,volumeId:%d, filename:%s", vid, dataBaseFileName) glog.Warningf("vif file not found,volumeId:%d, filename:%s", vid, dataBaseFileName)
volume_info.SaveVolumeInfo(dataBaseFileName+".vif", &volume_server_pb.VolumeInfo{Version: uint32(ev.Version)}) volume_info.SaveVolumeInfo(dataBaseFileName+".vif", &volume_server_pb.VolumeInfo{Version: uint32(ev.Version)})
ev.ECContext = NewDefaultECContext(collection, vid)
} }
ev.ShardLocations = make(map[ShardId][]pb.ServerAddress) ev.ShardLocations = make(map[ShardId][]pb.ServerAddress)
@ -260,7 +284,7 @@ func (ev *EcVolume) LocateEcShardNeedleInterval(version needle.Version, offset i
if ev.datFileSize > 0 { if ev.datFileSize > 0 {
// To get the correct LargeBlockRowsCount // To get the correct LargeBlockRowsCount
// use datFileSize to calculate the shardSize to match the EC encoding logic. // use datFileSize to calculate the shardSize to match the EC encoding logic.
shardSize = ev.datFileSize / DataShardsCount
shardSize = ev.datFileSize / int64(ev.ECContext.DataShards)
} }
// calculate the locations in the ec shards // calculate the locations in the ec shards
intervals = LocateData(ErasureCodingLargeBlockSize, ErasureCodingSmallBlockSize, shardSize, offset, types.Size(needle.GetActualSize(size, version))) intervals = LocateData(ErasureCodingLargeBlockSize, ErasureCodingSmallBlockSize, shardSize, offset, types.Size(needle.GetActualSize(size, version)))

21
weed/storage/erasure_coding/ec_volume_info.go

@ -87,7 +87,7 @@ func (ecInfo *EcVolumeInfo) Minus(other *EcVolumeInfo) *EcVolumeInfo {
// Copy shard sizes for remaining shards // Copy shard sizes for remaining shards
retIndex := 0 retIndex := 0
for shardId := ShardId(0); shardId < TotalShardsCount && retIndex < len(ret.ShardSizes); shardId++ {
for shardId := ShardId(0); shardId < ShardId(MaxShardCount) && retIndex < len(ret.ShardSizes); shardId++ {
if ret.ShardBits.HasShardId(shardId) { if ret.ShardBits.HasShardId(shardId) {
if size, exists := ecInfo.GetShardSize(shardId); exists { if size, exists := ecInfo.GetShardSize(shardId); exists {
ret.ShardSizes[retIndex] = size ret.ShardSizes[retIndex] = size
@ -119,19 +119,28 @@ func (ecInfo *EcVolumeInfo) ToVolumeEcShardInformationMessage() (ret *master_pb.
type ShardBits uint32 // use bits to indicate the shard id, use 32 bits just for possible future extension type ShardBits uint32 // use bits to indicate the shard id, use 32 bits just for possible future extension
func (b ShardBits) AddShardId(id ShardId) ShardBits { func (b ShardBits) AddShardId(id ShardId) ShardBits {
if id >= MaxShardCount {
return b // Reject out-of-range shard IDs
}
return b | (1 << id) return b | (1 << id)
} }
func (b ShardBits) RemoveShardId(id ShardId) ShardBits { func (b ShardBits) RemoveShardId(id ShardId) ShardBits {
if id >= MaxShardCount {
return b // Reject out-of-range shard IDs
}
return b &^ (1 << id) return b &^ (1 << id)
} }
func (b ShardBits) HasShardId(id ShardId) bool { func (b ShardBits) HasShardId(id ShardId) bool {
if id >= MaxShardCount {
return false // Out-of-range shard IDs are never present
}
return b&(1<<id) > 0 return b&(1<<id) > 0
} }
func (b ShardBits) ShardIds() (ret []ShardId) { func (b ShardBits) ShardIds() (ret []ShardId) {
for i := ShardId(0); i < TotalShardsCount; i++ {
for i := ShardId(0); i < ShardId(MaxShardCount); i++ {
if b.HasShardId(i) { if b.HasShardId(i) {
ret = append(ret, i) ret = append(ret, i)
} }
@ -140,7 +149,7 @@ func (b ShardBits) ShardIds() (ret []ShardId) {
} }
func (b ShardBits) ToUint32Slice() (ret []uint32) { func (b ShardBits) ToUint32Slice() (ret []uint32) {
for i := uint32(0); i < TotalShardsCount; i++ {
for i := uint32(0); i < uint32(MaxShardCount); i++ {
if b.HasShardId(ShardId(i)) { if b.HasShardId(ShardId(i)) {
ret = append(ret, i) ret = append(ret, i)
} }
@ -164,6 +173,8 @@ func (b ShardBits) Plus(other ShardBits) ShardBits {
} }
func (b ShardBits) MinusParityShards() ShardBits { func (b ShardBits) MinusParityShards() ShardBits {
// Removes parity shards from the bit mask
// Assumes default 10+4 EC layout where parity shards are IDs 10-13
for i := DataShardsCount; i < TotalShardsCount; i++ { for i := DataShardsCount; i < TotalShardsCount; i++ {
b = b.RemoveShardId(ShardId(i)) b = b.RemoveShardId(ShardId(i))
} }
@ -205,7 +216,7 @@ func (b ShardBits) IndexToShardId(index int) (shardId ShardId, found bool) {
} }
currentIndex := 0 currentIndex := 0
for i := ShardId(0); i < TotalShardsCount; i++ {
for i := ShardId(0); i < ShardId(MaxShardCount); i++ {
if b.HasShardId(i) { if b.HasShardId(i) {
if currentIndex == index { if currentIndex == index {
return i, true return i, true
@ -234,7 +245,7 @@ func (ecInfo *EcVolumeInfo) resizeShardSizes(prevShardBits ShardBits) {
// Copy existing sizes to new positions based on current ShardBits // Copy existing sizes to new positions based on current ShardBits
if len(ecInfo.ShardSizes) > 0 { if len(ecInfo.ShardSizes) > 0 {
newIndex := 0 newIndex := 0
for shardId := ShardId(0); shardId < TotalShardsCount && newIndex < expectedLength; shardId++ {
for shardId := ShardId(0); shardId < ShardId(MaxShardCount) && newIndex < expectedLength; shardId++ {
if ecInfo.ShardBits.HasShardId(shardId) { if ecInfo.ShardBits.HasShardId(shardId) {
// Try to find the size for this shard in the old array using previous ShardBits // Try to find the size for this shard in the old array using previous ShardBits
if oldIndex, found := prevShardBits.ShardIdToIndex(shardId); found && oldIndex < len(ecInfo.ShardSizes) { if oldIndex, found := prevShardBits.ShardIdToIndex(shardId); found && oldIndex < len(ecInfo.ShardSizes) {

3
weed/storage/store_ec.go

@ -350,7 +350,8 @@ func (s *Store) recoverOneRemoteEcShardInterval(needleId types.NeedleId, ecVolum
return 0, false, fmt.Errorf("failed to create encoder: %w", err) return 0, false, fmt.Errorf("failed to create encoder: %w", err)
} }
bufs := make([][]byte, erasure_coding.TotalShardsCount)
// Use MaxShardCount to support custom EC ratios up to 32 shards
bufs := make([][]byte, erasure_coding.MaxShardCount)
var wg sync.WaitGroup var wg sync.WaitGroup
ecVolume.ShardLocationsLock.RLock() ecVolume.ShardLocationsLock.RLock()

11
weed/topology/topology_ec.go

@ -10,7 +10,8 @@ import (
type EcShardLocations struct { type EcShardLocations struct {
Collection string Collection string
Locations [erasure_coding.TotalShardsCount][]*DataNode
// Use MaxShardCount (32) to support custom EC ratios
Locations [erasure_coding.MaxShardCount][]*DataNode
} }
func (t *Topology) SyncDataNodeEcShards(shardInfos []*master_pb.VolumeEcShardInformationMessage, dn *DataNode) (newShards, deletedShards []*erasure_coding.EcVolumeInfo) { func (t *Topology) SyncDataNodeEcShards(shardInfos []*master_pb.VolumeEcShardInformationMessage, dn *DataNode) (newShards, deletedShards []*erasure_coding.EcVolumeInfo) {
@ -90,6 +91,10 @@ func NewEcShardLocations(collection string) *EcShardLocations {
} }
func (loc *EcShardLocations) AddShard(shardId erasure_coding.ShardId, dn *DataNode) (added bool) { func (loc *EcShardLocations) AddShard(shardId erasure_coding.ShardId, dn *DataNode) (added bool) {
// Defensive bounds check to prevent panic with out-of-range shard IDs
if int(shardId) >= erasure_coding.MaxShardCount {
return false
}
dataNodes := loc.Locations[shardId] dataNodes := loc.Locations[shardId]
for _, n := range dataNodes { for _, n := range dataNodes {
if n.Id() == dn.Id() { if n.Id() == dn.Id() {
@ -101,6 +106,10 @@ func (loc *EcShardLocations) AddShard(shardId erasure_coding.ShardId, dn *DataNo
} }
func (loc *EcShardLocations) DeleteShard(shardId erasure_coding.ShardId, dn *DataNode) (deleted bool) { func (loc *EcShardLocations) DeleteShard(shardId erasure_coding.ShardId, dn *DataNode) (deleted bool) {
// Defensive bounds check to prevent panic with out-of-range shard IDs
if int(shardId) >= erasure_coding.MaxShardCount {
return false
}
dataNodes := loc.Locations[shardId] dataNodes := loc.Locations[shardId]
foundIndex := -1 foundIndex := -1
for index, n := range dataNodes { for index, n := range dataNodes {

202
weed/util/log_buffer/log_buffer_flush_gap_test.go

@ -15,10 +15,11 @@ import (
// are lost in the gap between flushed disk data and in-memory buffer. // are lost in the gap between flushed disk data and in-memory buffer.
// //
// OBSERVED BEHAVIOR FROM LOGS: // OBSERVED BEHAVIOR FROM LOGS:
// Request offset: 1764
// Disk contains: 1000-1763 (764 messages)
// Memory buffer starts at: 1800
// Gap: 1764-1799 (36 messages) ← MISSING!
//
// Request offset: 1764
// Disk contains: 1000-1763 (764 messages)
// Memory buffer starts at: 1800
// Gap: 1764-1799 (36 messages) ← MISSING!
// //
// This test verifies: // This test verifies:
// 1. All messages sent to buffer are accounted for // 1. All messages sent to buffer are accounted for
@ -27,46 +28,46 @@ import (
func TestFlushOffsetGap_ReproduceDataLoss(t *testing.T) { func TestFlushOffsetGap_ReproduceDataLoss(t *testing.T) {
var flushedMessages []*filer_pb.LogEntry var flushedMessages []*filer_pb.LogEntry
var flushMu sync.Mutex var flushMu sync.Mutex
flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) { flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) {
t.Logf("FLUSH: minOffset=%d maxOffset=%d size=%d bytes", minOffset, maxOffset, len(buf)) t.Logf("FLUSH: minOffset=%d maxOffset=%d size=%d bytes", minOffset, maxOffset, len(buf))
// Parse and store flushed messages // Parse and store flushed messages
flushMu.Lock() flushMu.Lock()
defer flushMu.Unlock() defer flushMu.Unlock()
// Parse buffer to extract messages // Parse buffer to extract messages
parsedCount := 0 parsedCount := 0
for pos := 0; pos+4 < len(buf); { for pos := 0; pos+4 < len(buf); {
if pos+4 > len(buf) { if pos+4 > len(buf) {
break break
} }
size := uint32(buf[pos])<<24 | uint32(buf[pos+1])<<16 | uint32(buf[pos+2])<<8 | uint32(buf[pos+3]) size := uint32(buf[pos])<<24 | uint32(buf[pos+1])<<16 | uint32(buf[pos+2])<<8 | uint32(buf[pos+3])
if pos+4+int(size) > len(buf) { if pos+4+int(size) > len(buf) {
break break
} }
entryData := buf[pos+4 : pos+4+int(size)] entryData := buf[pos+4 : pos+4+int(size)]
logEntry := &filer_pb.LogEntry{} logEntry := &filer_pb.LogEntry{}
if err := proto.Unmarshal(entryData, logEntry); err == nil { if err := proto.Unmarshal(entryData, logEntry); err == nil {
flushedMessages = append(flushedMessages, logEntry) flushedMessages = append(flushedMessages, logEntry)
parsedCount++ parsedCount++
} }
pos += 4 + int(size) pos += 4 + int(size)
} }
t.Logf(" Parsed %d messages from flush buffer", parsedCount) t.Logf(" Parsed %d messages from flush buffer", parsedCount)
} }
logBuffer := NewLogBuffer("test", 100*time.Millisecond, flushFn, nil, nil) logBuffer := NewLogBuffer("test", 100*time.Millisecond, flushFn, nil, nil)
defer logBuffer.ShutdownLogBuffer() defer logBuffer.ShutdownLogBuffer()
// Send 100 messages // Send 100 messages
messageCount := 100 messageCount := 100
t.Logf("Sending %d messages...", messageCount) t.Logf("Sending %d messages...", messageCount)
for i := 0; i < messageCount; i++ { for i := 0; i < messageCount; i++ {
logBuffer.AddToBuffer(&mq_pb.DataMessage{ logBuffer.AddToBuffer(&mq_pb.DataMessage{
Key: []byte(fmt.Sprintf("key-%d", i)), Key: []byte(fmt.Sprintf("key-%d", i)),
@ -74,11 +75,11 @@ func TestFlushOffsetGap_ReproduceDataLoss(t *testing.T) {
TsNs: time.Now().UnixNano(), TsNs: time.Now().UnixNano(),
}) })
} }
// Force flush multiple times to simulate real workload // Force flush multiple times to simulate real workload
t.Logf("Forcing flush...") t.Logf("Forcing flush...")
logBuffer.ForceFlush() logBuffer.ForceFlush()
// Add more messages after flush // Add more messages after flush
for i := messageCount; i < messageCount+50; i++ { for i := messageCount; i < messageCount+50; i++ {
logBuffer.AddToBuffer(&mq_pb.DataMessage{ logBuffer.AddToBuffer(&mq_pb.DataMessage{
@ -87,18 +88,18 @@ func TestFlushOffsetGap_ReproduceDataLoss(t *testing.T) {
TsNs: time.Now().UnixNano(), TsNs: time.Now().UnixNano(),
}) })
} }
// Force another flush // Force another flush
logBuffer.ForceFlush() logBuffer.ForceFlush()
time.Sleep(200 * time.Millisecond) // Wait for flush to complete time.Sleep(200 * time.Millisecond) // Wait for flush to complete
// Now check the buffer state // Now check the buffer state
logBuffer.RLock() logBuffer.RLock()
bufferStartOffset := logBuffer.bufferStartOffset bufferStartOffset := logBuffer.bufferStartOffset
currentOffset := logBuffer.offset currentOffset := logBuffer.offset
pos := logBuffer.pos pos := logBuffer.pos
logBuffer.RUnlock() logBuffer.RUnlock()
flushMu.Lock() flushMu.Lock()
flushedCount := len(flushedMessages) flushedCount := len(flushedMessages)
var maxFlushedOffset int64 = -1 var maxFlushedOffset int64 = -1
@ -108,23 +109,23 @@ func TestFlushOffsetGap_ReproduceDataLoss(t *testing.T) {
maxFlushedOffset = flushedMessages[flushedCount-1].Offset maxFlushedOffset = flushedMessages[flushedCount-1].Offset
} }
flushMu.Unlock() flushMu.Unlock()
t.Logf("\nBUFFER STATE AFTER FLUSH:") t.Logf("\nBUFFER STATE AFTER FLUSH:")
t.Logf(" bufferStartOffset: %d", bufferStartOffset) t.Logf(" bufferStartOffset: %d", bufferStartOffset)
t.Logf(" currentOffset (HWM): %d", currentOffset) t.Logf(" currentOffset (HWM): %d", currentOffset)
t.Logf(" pos (bytes in buffer): %d", pos) t.Logf(" pos (bytes in buffer): %d", pos)
t.Logf(" Messages sent: %d (offsets 0-%d)", messageCount+50, messageCount+49) t.Logf(" Messages sent: %d (offsets 0-%d)", messageCount+50, messageCount+49)
t.Logf(" Messages flushed to disk: %d (offsets %d-%d)", flushedCount, minFlushedOffset, maxFlushedOffset) t.Logf(" Messages flushed to disk: %d (offsets %d-%d)", flushedCount, minFlushedOffset, maxFlushedOffset)
// CRITICAL CHECK: Is there a gap between flushed data and memory buffer? // CRITICAL CHECK: Is there a gap between flushed data and memory buffer?
if flushedCount > 0 && maxFlushedOffset >= 0 { if flushedCount > 0 && maxFlushedOffset >= 0 {
gap := bufferStartOffset - (maxFlushedOffset + 1) gap := bufferStartOffset - (maxFlushedOffset + 1)
t.Logf("\nOFFSET CONTINUITY CHECK:") t.Logf("\nOFFSET CONTINUITY CHECK:")
t.Logf(" Last flushed offset: %d", maxFlushedOffset) t.Logf(" Last flushed offset: %d", maxFlushedOffset)
t.Logf(" Buffer starts at: %d", bufferStartOffset) t.Logf(" Buffer starts at: %d", bufferStartOffset)
t.Logf(" Gap: %d offsets", gap) t.Logf(" Gap: %d offsets", gap)
if gap > 0 { if gap > 0 {
t.Errorf("❌ CRITICAL BUG REPRODUCED: OFFSET GAP DETECTED!") t.Errorf("❌ CRITICAL BUG REPRODUCED: OFFSET GAP DETECTED!")
t.Errorf(" Disk has offsets %d-%d", minFlushedOffset, maxFlushedOffset) t.Errorf(" Disk has offsets %d-%d", minFlushedOffset, maxFlushedOffset)
@ -137,22 +138,22 @@ func TestFlushOffsetGap_ReproduceDataLoss(t *testing.T) {
} else { } else {
t.Logf("✅ PASS: No gap detected - offsets are continuous") t.Logf("✅ PASS: No gap detected - offsets are continuous")
} }
// Check if we can read all expected offsets // Check if we can read all expected offsets
t.Logf("\nREADABILITY CHECK:") t.Logf("\nREADABILITY CHECK:")
for testOffset := int64(0); testOffset < currentOffset; testOffset += 10 { for testOffset := int64(0); testOffset < currentOffset; testOffset += 10 {
// Try to read from buffer // Try to read from buffer
requestPosition := NewMessagePositionFromOffset(testOffset) requestPosition := NewMessagePositionFromOffset(testOffset)
buf, _, err := logBuffer.ReadFromBuffer(requestPosition) buf, _, err := logBuffer.ReadFromBuffer(requestPosition)
isReadable := (buf != nil && len(buf.Bytes()) > 0) || err == ResumeFromDiskError isReadable := (buf != nil && len(buf.Bytes()) > 0) || err == ResumeFromDiskError
status := "✅" status := "✅"
if !isReadable && err == nil { if !isReadable && err == nil {
status = "❌ NOT READABLE" status = "❌ NOT READABLE"
} }
t.Logf(" Offset %d: %s (buf=%v, err=%v)", testOffset, status, buf != nil, err) t.Logf(" Offset %d: %s (buf=%v, err=%v)", testOffset, status, buf != nil, err)
// If offset is in the gap, it should fail to read // If offset is in the gap, it should fail to read
if flushedCount > 0 && testOffset > maxFlushedOffset && testOffset < bufferStartOffset { if flushedCount > 0 && testOffset > maxFlushedOffset && testOffset < bufferStartOffset {
if isReadable { if isReadable {
@ -163,19 +164,19 @@ func TestFlushOffsetGap_ReproduceDataLoss(t *testing.T) {
} }
} }
} }
// Check that all sent messages are accounted for // Check that all sent messages are accounted for
expectedMessageCount := messageCount + 50 expectedMessageCount := messageCount + 50
messagesInMemory := int(currentOffset - bufferStartOffset) messagesInMemory := int(currentOffset - bufferStartOffset)
totalAccountedFor := flushedCount + messagesInMemory totalAccountedFor := flushedCount + messagesInMemory
t.Logf("\nMESSAGE ACCOUNTING:") t.Logf("\nMESSAGE ACCOUNTING:")
t.Logf(" Expected: %d messages", expectedMessageCount) t.Logf(" Expected: %d messages", expectedMessageCount)
t.Logf(" Flushed to disk: %d", flushedCount) t.Logf(" Flushed to disk: %d", flushedCount)
t.Logf(" In memory buffer: %d (offset range %d-%d)", messagesInMemory, bufferStartOffset, currentOffset-1) t.Logf(" In memory buffer: %d (offset range %d-%d)", messagesInMemory, bufferStartOffset, currentOffset-1)
t.Logf(" Total accounted for: %d", totalAccountedFor) t.Logf(" Total accounted for: %d", totalAccountedFor)
t.Logf(" Missing: %d messages", expectedMessageCount-totalAccountedFor) t.Logf(" Missing: %d messages", expectedMessageCount-totalAccountedFor)
if totalAccountedFor < expectedMessageCount { if totalAccountedFor < expectedMessageCount {
t.Errorf("❌ DATA LOSS CONFIRMED: %d messages are missing!", expectedMessageCount-totalAccountedFor) t.Errorf("❌ DATA LOSS CONFIRMED: %d messages are missing!", expectedMessageCount-totalAccountedFor)
} else { } else {
@ -188,23 +189,23 @@ func TestFlushOffsetGap_ReproduceDataLoss(t *testing.T) {
func TestFlushOffsetGap_CheckPrevBuffers(t *testing.T) { func TestFlushOffsetGap_CheckPrevBuffers(t *testing.T) {
var flushCount int var flushCount int
var flushMu sync.Mutex var flushMu sync.Mutex
flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) { flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) {
flushMu.Lock() flushMu.Lock()
flushCount++ flushCount++
count := flushCount count := flushCount
flushMu.Unlock() flushMu.Unlock()
t.Logf("FLUSH #%d: minOffset=%d maxOffset=%d size=%d bytes", count, minOffset, maxOffset, len(buf)) t.Logf("FLUSH #%d: minOffset=%d maxOffset=%d size=%d bytes", count, minOffset, maxOffset, len(buf))
} }
logBuffer := NewLogBuffer("test", 100*time.Millisecond, flushFn, nil, nil) logBuffer := NewLogBuffer("test", 100*time.Millisecond, flushFn, nil, nil)
defer logBuffer.ShutdownLogBuffer() defer logBuffer.ShutdownLogBuffer()
// Send messages in batches with flushes in between // Send messages in batches with flushes in between
for batch := 0; batch < 5; batch++ { for batch := 0; batch < 5; batch++ {
t.Logf("\nBatch %d:", batch) t.Logf("\nBatch %d:", batch)
// Send 20 messages // Send 20 messages
for i := 0; i < 20; i++ { for i := 0; i < 20; i++ {
offset := int64(batch*20 + i) offset := int64(batch*20 + i)
@ -214,28 +215,28 @@ func TestFlushOffsetGap_CheckPrevBuffers(t *testing.T) {
TsNs: time.Now().UnixNano(), TsNs: time.Now().UnixNano(),
}) })
} }
// Check state before flush // Check state before flush
logBuffer.RLock() logBuffer.RLock()
beforeFlushOffset := logBuffer.offset beforeFlushOffset := logBuffer.offset
beforeFlushStart := logBuffer.bufferStartOffset beforeFlushStart := logBuffer.bufferStartOffset
logBuffer.RUnlock() logBuffer.RUnlock()
// Force flush // Force flush
logBuffer.ForceFlush() logBuffer.ForceFlush()
time.Sleep(50 * time.Millisecond) time.Sleep(50 * time.Millisecond)
// Check state after flush // Check state after flush
logBuffer.RLock() logBuffer.RLock()
afterFlushOffset := logBuffer.offset afterFlushOffset := logBuffer.offset
afterFlushStart := logBuffer.bufferStartOffset afterFlushStart := logBuffer.bufferStartOffset
prevBufferCount := len(logBuffer.prevBuffers.buffers) prevBufferCount := len(logBuffer.prevBuffers.buffers)
// Check prevBuffers state // Check prevBuffers state
t.Logf(" Before flush: offset=%d, bufferStartOffset=%d", beforeFlushOffset, beforeFlushStart) t.Logf(" Before flush: offset=%d, bufferStartOffset=%d", beforeFlushOffset, beforeFlushStart)
t.Logf(" After flush: offset=%d, bufferStartOffset=%d, prevBuffers=%d", t.Logf(" After flush: offset=%d, bufferStartOffset=%d, prevBuffers=%d",
afterFlushOffset, afterFlushStart, prevBufferCount) afterFlushOffset, afterFlushStart, prevBufferCount)
// Check each prevBuffer // Check each prevBuffer
for i, prevBuf := range logBuffer.prevBuffers.buffers { for i, prevBuf := range logBuffer.prevBuffers.buffers {
if prevBuf.size > 0 { if prevBuf.size > 0 {
@ -244,7 +245,7 @@ func TestFlushOffsetGap_CheckPrevBuffers(t *testing.T) {
} }
} }
logBuffer.RUnlock() logBuffer.RUnlock()
// CRITICAL: Check if bufferStartOffset advanced correctly // CRITICAL: Check if bufferStartOffset advanced correctly
expectedNewStart := beforeFlushOffset expectedNewStart := beforeFlushOffset
if afterFlushStart != expectedNewStart { if afterFlushStart != expectedNewStart {
@ -261,10 +262,10 @@ func TestFlushOffsetGap_CheckPrevBuffers(t *testing.T) {
func TestFlushOffsetGap_ConcurrentWriteAndFlush(t *testing.T) { func TestFlushOffsetGap_ConcurrentWriteAndFlush(t *testing.T) {
var allFlushedOffsets []int64 var allFlushedOffsets []int64
var flushMu sync.Mutex var flushMu sync.Mutex
flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) { flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) {
t.Logf("FLUSH: offsets %d-%d (%d bytes)", minOffset, maxOffset, len(buf)) t.Logf("FLUSH: offsets %d-%d (%d bytes)", minOffset, maxOffset, len(buf))
flushMu.Lock() flushMu.Lock()
// Record the offset range that was flushed // Record the offset range that was flushed
for offset := minOffset; offset <= maxOffset; offset++ { for offset := minOffset; offset <= maxOffset; offset++ {
@ -272,13 +273,13 @@ func TestFlushOffsetGap_ConcurrentWriteAndFlush(t *testing.T) {
} }
flushMu.Unlock() flushMu.Unlock()
} }
logBuffer := NewLogBuffer("test", 50*time.Millisecond, flushFn, nil, nil) logBuffer := NewLogBuffer("test", 50*time.Millisecond, flushFn, nil, nil)
defer logBuffer.ShutdownLogBuffer() defer logBuffer.ShutdownLogBuffer()
// Concurrently write messages and force flushes // Concurrently write messages and force flushes
var wg sync.WaitGroup var wg sync.WaitGroup
// Writer goroutine // Writer goroutine
wg.Add(1) wg.Add(1)
go func() { go func() {
@ -294,7 +295,7 @@ func TestFlushOffsetGap_ConcurrentWriteAndFlush(t *testing.T) {
} }
} }
}() }()
// Flusher goroutine // Flusher goroutine
wg.Add(1) wg.Add(1)
go func() { go func() {
@ -304,31 +305,31 @@ func TestFlushOffsetGap_ConcurrentWriteAndFlush(t *testing.T) {
logBuffer.ForceFlush() logBuffer.ForceFlush()
} }
}() }()
wg.Wait() wg.Wait()
time.Sleep(200 * time.Millisecond) // Wait for final flush time.Sleep(200 * time.Millisecond) // Wait for final flush
// Check final state // Check final state
logBuffer.RLock() logBuffer.RLock()
finalOffset := logBuffer.offset finalOffset := logBuffer.offset
finalBufferStart := logBuffer.bufferStartOffset finalBufferStart := logBuffer.bufferStartOffset
logBuffer.RUnlock() logBuffer.RUnlock()
flushMu.Lock() flushMu.Lock()
flushedCount := len(allFlushedOffsets) flushedCount := len(allFlushedOffsets)
flushMu.Unlock() flushMu.Unlock()
expectedCount := int(finalOffset) expectedCount := int(finalOffset)
inMemory := int(finalOffset - finalBufferStart) inMemory := int(finalOffset - finalBufferStart)
totalAccountedFor := flushedCount + inMemory totalAccountedFor := flushedCount + inMemory
t.Logf("\nFINAL STATE:") t.Logf("\nFINAL STATE:")
t.Logf(" Total messages sent: %d (offsets 0-%d)", expectedCount, expectedCount-1) t.Logf(" Total messages sent: %d (offsets 0-%d)", expectedCount, expectedCount-1)
t.Logf(" Flushed to disk: %d", flushedCount) t.Logf(" Flushed to disk: %d", flushedCount)
t.Logf(" In memory: %d (offsets %d-%d)", inMemory, finalBufferStart, finalOffset-1) t.Logf(" In memory: %d (offsets %d-%d)", inMemory, finalBufferStart, finalOffset-1)
t.Logf(" Total accounted: %d", totalAccountedFor) t.Logf(" Total accounted: %d", totalAccountedFor)
t.Logf(" Missing: %d", expectedCount-totalAccountedFor) t.Logf(" Missing: %d", expectedCount-totalAccountedFor)
if totalAccountedFor < expectedCount { if totalAccountedFor < expectedCount {
t.Errorf("❌ DATA LOSS in concurrent scenario: %d messages missing!", expectedCount-totalAccountedFor) t.Errorf("❌ DATA LOSS in concurrent scenario: %d messages missing!", expectedCount-totalAccountedFor)
} }
@ -344,7 +345,7 @@ func TestFlushOffsetGap_ProductionScenario(t *testing.T) {
messages []*filer_pb.LogEntry messages []*filer_pb.LogEntry
} }
var flushMu sync.Mutex var flushMu sync.Mutex
flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) { flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) {
// Parse messages from buffer // Parse messages from buffer
messages := []*filer_pb.LogEntry{} messages := []*filer_pb.LogEntry{}
@ -360,7 +361,7 @@ func TestFlushOffsetGap_ProductionScenario(t *testing.T) {
} }
pos += 4 + int(size) pos += 4 + int(size)
} }
flushMu.Lock() flushMu.Lock()
flushedData = append(flushedData, struct { flushedData = append(flushedData, struct {
minOffset int64 minOffset int64
@ -368,17 +369,17 @@ func TestFlushOffsetGap_ProductionScenario(t *testing.T) {
messages []*filer_pb.LogEntry messages []*filer_pb.LogEntry
}{minOffset, maxOffset, messages}) }{minOffset, maxOffset, messages})
flushMu.Unlock() flushMu.Unlock()
t.Logf("FLUSH: minOffset=%d maxOffset=%d, parsed %d messages", minOffset, maxOffset, len(messages)) t.Logf("FLUSH: minOffset=%d maxOffset=%d, parsed %d messages", minOffset, maxOffset, len(messages))
} }
logBuffer := NewLogBuffer("test", time.Hour, flushFn, nil, nil) logBuffer := NewLogBuffer("test", time.Hour, flushFn, nil, nil)
defer logBuffer.ShutdownLogBuffer() defer logBuffer.ShutdownLogBuffer()
// Simulate broker behavior: assign Kafka offsets and add to buffer // Simulate broker behavior: assign Kafka offsets and add to buffer
// This is what PublishWithOffset() does // This is what PublishWithOffset() does
nextKafkaOffset := int64(0) nextKafkaOffset := int64(0)
// Round 1: Add 50 messages with Kafka offsets 0-49 // Round 1: Add 50 messages with Kafka offsets 0-49
t.Logf("\n=== ROUND 1: Adding messages 0-49 ===") t.Logf("\n=== ROUND 1: Adding messages 0-49 ===")
for i := 0; i < 50; i++ { for i := 0; i < 50; i++ {
@ -391,7 +392,7 @@ func TestFlushOffsetGap_ProductionScenario(t *testing.T) {
logBuffer.AddLogEntryToBuffer(logEntry) logBuffer.AddLogEntryToBuffer(logEntry)
nextKafkaOffset++ nextKafkaOffset++
} }
// Check buffer state before flush // Check buffer state before flush
logBuffer.RLock() logBuffer.RLock()
beforeFlushOffset := logBuffer.offset beforeFlushOffset := logBuffer.offset
@ -399,11 +400,11 @@ func TestFlushOffsetGap_ProductionScenario(t *testing.T) {
logBuffer.RUnlock() logBuffer.RUnlock()
t.Logf("Before flush: logBuffer.offset=%d, bufferStartOffset=%d, nextKafkaOffset=%d", t.Logf("Before flush: logBuffer.offset=%d, bufferStartOffset=%d, nextKafkaOffset=%d",
beforeFlushOffset, beforeFlushStart, nextKafkaOffset) beforeFlushOffset, beforeFlushStart, nextKafkaOffset)
// Flush // Flush
logBuffer.ForceFlush() logBuffer.ForceFlush()
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
// Check buffer state after flush // Check buffer state after flush
logBuffer.RLock() logBuffer.RLock()
afterFlushOffset := logBuffer.offset afterFlushOffset := logBuffer.offset
@ -411,7 +412,7 @@ func TestFlushOffsetGap_ProductionScenario(t *testing.T) {
logBuffer.RUnlock() logBuffer.RUnlock()
t.Logf("After flush: logBuffer.offset=%d, bufferStartOffset=%d", t.Logf("After flush: logBuffer.offset=%d, bufferStartOffset=%d",
afterFlushOffset, afterFlushStart) afterFlushOffset, afterFlushStart)
// Round 2: Add another 50 messages with Kafka offsets 50-99 // Round 2: Add another 50 messages with Kafka offsets 50-99
t.Logf("\n=== ROUND 2: Adding messages 50-99 ===") t.Logf("\n=== ROUND 2: Adding messages 50-99 ===")
for i := 0; i < 50; i++ { for i := 0; i < 50; i++ {
@ -424,20 +425,20 @@ func TestFlushOffsetGap_ProductionScenario(t *testing.T) {
logBuffer.AddLogEntryToBuffer(logEntry) logBuffer.AddLogEntryToBuffer(logEntry)
nextKafkaOffset++ nextKafkaOffset++
} }
logBuffer.ForceFlush() logBuffer.ForceFlush()
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
// Verification: Check if all Kafka offsets are accounted for // Verification: Check if all Kafka offsets are accounted for
flushMu.Lock() flushMu.Lock()
t.Logf("\n=== VERIFICATION ===") t.Logf("\n=== VERIFICATION ===")
t.Logf("Expected Kafka offsets: 0-%d", nextKafkaOffset-1) t.Logf("Expected Kafka offsets: 0-%d", nextKafkaOffset-1)
allOffsets := make(map[int64]bool) allOffsets := make(map[int64]bool)
for flushIdx, flush := range flushedData { for flushIdx, flush := range flushedData {
t.Logf("Flush #%d: minOffset=%d, maxOffset=%d, messages=%d", t.Logf("Flush #%d: minOffset=%d, maxOffset=%d, messages=%d",
flushIdx, flush.minOffset, flush.maxOffset, len(flush.messages)) flushIdx, flush.minOffset, flush.maxOffset, len(flush.messages))
for _, msg := range flush.messages { for _, msg := range flush.messages {
if allOffsets[msg.Offset] { if allOffsets[msg.Offset] {
t.Errorf(" ❌ DUPLICATE: Offset %d appears multiple times!", msg.Offset) t.Errorf(" ❌ DUPLICATE: Offset %d appears multiple times!", msg.Offset)
@ -446,7 +447,7 @@ func TestFlushOffsetGap_ProductionScenario(t *testing.T) {
} }
} }
flushMu.Unlock() flushMu.Unlock()
// Check for missing offsets // Check for missing offsets
missingOffsets := []int64{} missingOffsets := []int64{}
for expectedOffset := int64(0); expectedOffset < nextKafkaOffset; expectedOffset++ { for expectedOffset := int64(0); expectedOffset < nextKafkaOffset; expectedOffset++ {
@ -454,7 +455,7 @@ func TestFlushOffsetGap_ProductionScenario(t *testing.T) {
missingOffsets = append(missingOffsets, expectedOffset) missingOffsets = append(missingOffsets, expectedOffset)
} }
} }
if len(missingOffsets) > 0 { if len(missingOffsets) > 0 {
t.Errorf("\n❌ MISSING OFFSETS DETECTED: %d offsets missing", len(missingOffsets)) t.Errorf("\n❌ MISSING OFFSETS DETECTED: %d offsets missing", len(missingOffsets))
if len(missingOffsets) <= 20 { if len(missingOffsets) <= 20 {
@ -466,18 +467,18 @@ func TestFlushOffsetGap_ProductionScenario(t *testing.T) {
} else { } else {
t.Logf("\n✅ SUCCESS: All %d Kafka offsets accounted for (0-%d)", nextKafkaOffset, nextKafkaOffset-1) t.Logf("\n✅ SUCCESS: All %d Kafka offsets accounted for (0-%d)", nextKafkaOffset, nextKafkaOffset-1)
} }
// Check buffer offset consistency // Check buffer offset consistency
logBuffer.RLock() logBuffer.RLock()
finalOffset := logBuffer.offset finalOffset := logBuffer.offset
finalBufferStart := logBuffer.bufferStartOffset finalBufferStart := logBuffer.bufferStartOffset
logBuffer.RUnlock() logBuffer.RUnlock()
t.Logf("\nFinal buffer state:") t.Logf("\nFinal buffer state:")
t.Logf(" logBuffer.offset: %d", finalOffset) t.Logf(" logBuffer.offset: %d", finalOffset)
t.Logf(" bufferStartOffset: %d", finalBufferStart) t.Logf(" bufferStartOffset: %d", finalBufferStart)
t.Logf(" Expected (nextKafkaOffset): %d", nextKafkaOffset) t.Logf(" Expected (nextKafkaOffset): %d", nextKafkaOffset)
if finalOffset != nextKafkaOffset { if finalOffset != nextKafkaOffset {
t.Errorf("❌ logBuffer.offset mismatch: expected %d, got %d", nextKafkaOffset, finalOffset) t.Errorf("❌ logBuffer.offset mismatch: expected %d, got %d", nextKafkaOffset, finalOffset)
} }
@ -488,12 +489,12 @@ func TestFlushOffsetGap_ProductionScenario(t *testing.T) {
func TestFlushOffsetGap_ConcurrentReadDuringFlush(t *testing.T) { func TestFlushOffsetGap_ConcurrentReadDuringFlush(t *testing.T) {
var flushedOffsets []int64 var flushedOffsets []int64
var flushMu sync.Mutex var flushMu sync.Mutex
readFromDiskFn := func(startPosition MessagePosition, stopTsNs int64, eachLogEntryFn EachLogEntryFuncType) (MessagePosition, bool, error) { readFromDiskFn := func(startPosition MessagePosition, stopTsNs int64, eachLogEntryFn EachLogEntryFuncType) (MessagePosition, bool, error) {
// Simulate reading from disk - return flushed offsets // Simulate reading from disk - return flushed offsets
flushMu.Lock() flushMu.Lock()
defer flushMu.Unlock() defer flushMu.Unlock()
for _, offset := range flushedOffsets { for _, offset := range flushedOffsets {
if offset >= startPosition.Offset { if offset >= startPosition.Offset {
logEntry := &filer_pb.LogEntry{ logEntry := &filer_pb.LogEntry{
@ -510,12 +511,12 @@ func TestFlushOffsetGap_ConcurrentReadDuringFlush(t *testing.T) {
} }
return startPosition, false, nil return startPosition, false, nil
} }
flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) { flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) {
// Parse and store flushed offsets // Parse and store flushed offsets
flushMu.Lock() flushMu.Lock()
defer flushMu.Unlock() defer flushMu.Unlock()
for pos := 0; pos+4 < len(buf); { for pos := 0; pos+4 < len(buf); {
size := uint32(buf[pos])<<24 | uint32(buf[pos+1])<<16 | uint32(buf[pos+2])<<8 | uint32(buf[pos+3]) size := uint32(buf[pos])<<24 | uint32(buf[pos+1])<<16 | uint32(buf[pos+2])<<8 | uint32(buf[pos+3])
if pos+4+int(size) > len(buf) { if pos+4+int(size) > len(buf) {
@ -528,14 +529,14 @@ func TestFlushOffsetGap_ConcurrentReadDuringFlush(t *testing.T) {
} }
pos += 4 + int(size) pos += 4 + int(size)
} }
t.Logf("FLUSH: Stored %d offsets to disk (minOffset=%d, maxOffset=%d)", t.Logf("FLUSH: Stored %d offsets to disk (minOffset=%d, maxOffset=%d)",
len(flushedOffsets), minOffset, maxOffset) len(flushedOffsets), minOffset, maxOffset)
} }
logBuffer := NewLogBuffer("test", time.Hour, flushFn, readFromDiskFn, nil) logBuffer := NewLogBuffer("test", time.Hour, flushFn, readFromDiskFn, nil)
defer logBuffer.ShutdownLogBuffer() defer logBuffer.ShutdownLogBuffer()
// Add 100 messages // Add 100 messages
t.Logf("Adding 100 messages...") t.Logf("Adding 100 messages...")
for i := int64(0); i < 100; i++ { for i := int64(0); i < 100; i++ {
@ -547,32 +548,32 @@ func TestFlushOffsetGap_ConcurrentReadDuringFlush(t *testing.T) {
} }
logBuffer.AddLogEntryToBuffer(logEntry) logBuffer.AddLogEntryToBuffer(logEntry)
} }
// Flush (moves data to disk) // Flush (moves data to disk)
t.Logf("Flushing...") t.Logf("Flushing...")
logBuffer.ForceFlush() logBuffer.ForceFlush()
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
// Now try to read all messages using ReadMessagesAtOffset // Now try to read all messages using ReadMessagesAtOffset
t.Logf("\nReading messages from offset 0...") t.Logf("\nReading messages from offset 0...")
messages, nextOffset, hwm, endOfPartition, err := logBuffer.ReadMessagesAtOffset(0, 1000, 1024*1024) messages, nextOffset, hwm, endOfPartition, err := logBuffer.ReadMessagesAtOffset(0, 1000, 1024*1024)
t.Logf("Read result: messages=%d, nextOffset=%d, hwm=%d, endOfPartition=%v, err=%v", t.Logf("Read result: messages=%d, nextOffset=%d, hwm=%d, endOfPartition=%v, err=%v",
len(messages), nextOffset, hwm, endOfPartition, err) len(messages), nextOffset, hwm, endOfPartition, err)
// Verify all offsets can be read // Verify all offsets can be read
readOffsets := make(map[int64]bool) readOffsets := make(map[int64]bool)
for _, msg := range messages { for _, msg := range messages {
readOffsets[msg.Offset] = true readOffsets[msg.Offset] = true
} }
missingOffsets := []int64{} missingOffsets := []int64{}
for expectedOffset := int64(0); expectedOffset < 100; expectedOffset++ { for expectedOffset := int64(0); expectedOffset < 100; expectedOffset++ {
if !readOffsets[expectedOffset] { if !readOffsets[expectedOffset] {
missingOffsets = append(missingOffsets, expectedOffset) missingOffsets = append(missingOffsets, expectedOffset)
} }
} }
if len(missingOffsets) > 0 { if len(missingOffsets) > 0 {
t.Errorf("❌ MISSING OFFSETS after flush: %d offsets cannot be read", len(missingOffsets)) t.Errorf("❌ MISSING OFFSETS after flush: %d offsets cannot be read", len(missingOffsets))
if len(missingOffsets) <= 20 { if len(missingOffsets) <= 20 {
@ -590,29 +591,29 @@ func TestFlushOffsetGap_ConcurrentReadDuringFlush(t *testing.T) {
func TestFlushOffsetGap_ForceFlushAdvancesBuffer(t *testing.T) { func TestFlushOffsetGap_ForceFlushAdvancesBuffer(t *testing.T) {
flushedRanges := []struct{ min, max int64 }{} flushedRanges := []struct{ min, max int64 }{}
var flushMu sync.Mutex var flushMu sync.Mutex
flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) { flushFn := func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte, minOffset, maxOffset int64) {
flushMu.Lock() flushMu.Lock()
flushedRanges = append(flushedRanges, struct{ min, max int64 }{minOffset, maxOffset}) flushedRanges = append(flushedRanges, struct{ min, max int64 }{minOffset, maxOffset})
flushMu.Unlock() flushMu.Unlock()
t.Logf("FLUSH: offsets %d-%d", minOffset, maxOffset) t.Logf("FLUSH: offsets %d-%d", minOffset, maxOffset)
} }
logBuffer := NewLogBuffer("test", time.Hour, flushFn, nil, nil) // Long interval, manual flush only logBuffer := NewLogBuffer("test", time.Hour, flushFn, nil, nil) // Long interval, manual flush only
defer logBuffer.ShutdownLogBuffer() defer logBuffer.ShutdownLogBuffer()
// Send messages, flush, check state - repeat // Send messages, flush, check state - repeat
for round := 0; round < 3; round++ { for round := 0; round < 3; round++ {
t.Logf("\n=== ROUND %d ===", round) t.Logf("\n=== ROUND %d ===", round)
// Check state before adding messages // Check state before adding messages
logBuffer.RLock() logBuffer.RLock()
beforeOffset := logBuffer.offset beforeOffset := logBuffer.offset
beforeStart := logBuffer.bufferStartOffset beforeStart := logBuffer.bufferStartOffset
logBuffer.RUnlock() logBuffer.RUnlock()
t.Logf("Before adding: offset=%d, bufferStartOffset=%d", beforeOffset, beforeStart) t.Logf("Before adding: offset=%d, bufferStartOffset=%d", beforeOffset, beforeStart)
// Add 10 messages // Add 10 messages
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
logBuffer.AddToBuffer(&mq_pb.DataMessage{ logBuffer.AddToBuffer(&mq_pb.DataMessage{
@ -621,28 +622,28 @@ func TestFlushOffsetGap_ForceFlushAdvancesBuffer(t *testing.T) {
TsNs: time.Now().UnixNano(), TsNs: time.Now().UnixNano(),
}) })
} }
// Check state after adding // Check state after adding
logBuffer.RLock() logBuffer.RLock()
afterAddOffset := logBuffer.offset afterAddOffset := logBuffer.offset
afterAddStart := logBuffer.bufferStartOffset afterAddStart := logBuffer.bufferStartOffset
logBuffer.RUnlock() logBuffer.RUnlock()
t.Logf("After adding: offset=%d, bufferStartOffset=%d", afterAddOffset, afterAddStart) t.Logf("After adding: offset=%d, bufferStartOffset=%d", afterAddOffset, afterAddStart)
// Force flush // Force flush
t.Logf("Forcing flush...") t.Logf("Forcing flush...")
logBuffer.ForceFlush() logBuffer.ForceFlush()
time.Sleep(100 * time.Millisecond) time.Sleep(100 * time.Millisecond)
// Check state after flush // Check state after flush
logBuffer.RLock() logBuffer.RLock()
afterFlushOffset := logBuffer.offset afterFlushOffset := logBuffer.offset
afterFlushStart := logBuffer.bufferStartOffset afterFlushStart := logBuffer.bufferStartOffset
logBuffer.RUnlock() logBuffer.RUnlock()
t.Logf("After flush: offset=%d, bufferStartOffset=%d", afterFlushOffset, afterFlushStart) t.Logf("After flush: offset=%d, bufferStartOffset=%d", afterFlushOffset, afterFlushStart)
// CRITICAL CHECK: bufferStartOffset should advance to where offset was before flush // CRITICAL CHECK: bufferStartOffset should advance to where offset was before flush
if afterFlushStart != afterAddOffset { if afterFlushStart != afterAddOffset {
t.Errorf("❌ FLUSH BUG: bufferStartOffset did NOT advance correctly!") t.Errorf("❌ FLUSH BUG: bufferStartOffset did NOT advance correctly!")
@ -653,19 +654,19 @@ func TestFlushOffsetGap_ForceFlushAdvancesBuffer(t *testing.T) {
t.Logf("✅ bufferStartOffset correctly advanced to %d", afterFlushStart) t.Logf("✅ bufferStartOffset correctly advanced to %d", afterFlushStart)
} }
} }
// Final verification: check all offset ranges are continuous // Final verification: check all offset ranges are continuous
flushMu.Lock() flushMu.Lock()
t.Logf("\n=== FLUSHED RANGES ===") t.Logf("\n=== FLUSHED RANGES ===")
for i, r := range flushedRanges { for i, r := range flushedRanges {
t.Logf("Flush #%d: offsets %d-%d", i, r.min, r.max) t.Logf("Flush #%d: offsets %d-%d", i, r.min, r.max)
// Check continuity with previous flush // Check continuity with previous flush
if i > 0 { if i > 0 {
prevMax := flushedRanges[i-1].max prevMax := flushedRanges[i-1].max
currentMin := r.min currentMin := r.min
gap := currentMin - (prevMax + 1) gap := currentMin - (prevMax + 1)
if gap > 0 { if gap > 0 {
t.Errorf("❌ GAP between flush #%d and #%d: %d offsets missing!", i-1, i, gap) t.Errorf("❌ GAP between flush #%d and #%d: %d offsets missing!", i-1, i, gap)
} else if gap < 0 { } else if gap < 0 {
@ -677,4 +678,3 @@ func TestFlushOffsetGap_ForceFlushAdvancesBuffer(t *testing.T) {
} }
flushMu.Unlock() flushMu.Unlock()
} }

2
weed/util/version/constants.go

@ -8,7 +8,7 @@ import (
var ( var (
MAJOR_VERSION = int32(3) MAJOR_VERSION = int32(3)
MINOR_VERSION = int32(98)
MINOR_VERSION = int32(99)
VERSION_NUMBER = fmt.Sprintf("%d.%02d", MAJOR_VERSION, MINOR_VERSION) VERSION_NUMBER = fmt.Sprintf("%d.%02d", MAJOR_VERSION, MINOR_VERSION)
VERSION = util.SizeLimit + " " + VERSION_NUMBER VERSION = util.SizeLimit + " " + VERSION_NUMBER
COMMIT = "" COMMIT = ""

1
weed/worker/client.go

@ -18,7 +18,6 @@ type GrpcAdminClient struct {
adminAddress string adminAddress string
workerID string workerID string
dialOption grpc.DialOption dialOption grpc.DialOption
comms clientChannels comms clientChannels
// Reconnection parameters // Reconnection parameters

3
weed/worker/tasks/erasure_coding/ec_task.go

@ -374,7 +374,8 @@ func (t *ErasureCodingTask) generateEcShardsLocally(localFiles map[string]string
var generatedShards []string var generatedShards []string
var totalShardSize int64 var totalShardSize int64
for i := 0; i < erasure_coding.TotalShardsCount; i++ {
// Check up to MaxShardCount (32) to support custom EC ratios
for i := 0; i < erasure_coding.MaxShardCount; i++ {
shardFile := fmt.Sprintf("%s.ec%02d", baseName, i) shardFile := fmt.Sprintf("%s.ec%02d", baseName, i)
if info, err := os.Stat(shardFile); err == nil { if info, err := os.Stat(shardFile); err == nil {
shardKey := fmt.Sprintf("ec%02d", i) shardKey := fmt.Sprintf("ec%02d", i)

Loading…
Cancel
Save