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. 4
      weed/command/autocomplete.go
  7. 1
      weed/mq/broker/broker_grpc_sub.go
  8. 1
      weed/mq/kafka/consumer_offset/memory_storage.go
  9. 1
      weed/mq/kafka/consumer_offset/memory_storage_test.go
  10. 1
      weed/mq/kafka/consumer_offset/storage.go
  11. 1
      weed/mq/kafka/gateway/test_mock_handler.go
  12. 2
      weed/mq/kafka/package.go
  13. 2
      weed/mq/kafka/partition_mapping.go
  14. 2
      weed/mq/kafka/protocol/describe_cluster.go
  15. 1
      weed/mq/kafka/protocol/flexible_versions.go
  16. 5
      weed/mq/kafka/protocol/handler.go
  17. 1
      weed/mq/kafka/protocol/offset_storage_adapter.go
  18. 1
      weed/mq/kafka/protocol/response_validation_example_test.go
  19. 2
      weed/mq/metadata_constants.go
  20. 2
      weed/pb/mq_agent_pb/publish_response_test.go
  21. 2
      weed/pb/schema_pb/offset_test.go
  22. 7
      weed/pb/volume_server.proto
  23. 502
      weed/pb/volume_server_pb/volume_server.pb.go
  24. 64
      weed/server/volume_grpc_erasure_coding.go
  25. 14
      weed/shell/command_ec_common.go
  26. 3
      weed/shell/command_ec_rebuild.go
  27. 13
      weed/storage/disk_location_ec.go
  28. 46
      weed/storage/erasure_coding/ec_context.go
  29. 113
      weed/storage/erasure_coding/ec_encoder.go
  30. 19
      weed/storage/erasure_coding/ec_test.go
  31. 26
      weed/storage/erasure_coding/ec_volume.go
  32. 21
      weed/storage/erasure_coding/ec_volume_info.go
  33. 3
      weed/storage/store_ec.go
  34. 11
      weed/topology/topology_ec.go
  35. 2
      weed/util/log_buffer/log_buffer_flush_gap_test.go
  36. 2
      weed/util/version/constants.go
  37. 1
      weed/worker/client.go
  38. 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)
} }

4
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"
) )

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 {

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

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

@ -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")
@ -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)",
@ -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
@ -1994,7 +1991,6 @@ 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
@ -3930,7 +3926,6 @@ 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

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)
} }

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
) )

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) {

2
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) {

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

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")

26
weed/storage/erasure_coding/ec_volume.go

@ -42,6 +42,7 @@ type EcVolume struct {
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 {

2
weed/util/log_buffer/log_buffer_flush_gap_test.go

@ -15,6 +15,7 @@ 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 // Request offset: 1764
// Disk contains: 1000-1763 (764 messages) // Disk contains: 1000-1763 (764 messages)
// Memory buffer starts at: 1800 // Memory buffer starts at: 1800
@ -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