mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2025-11-24 08:46:54 +08:00
* refactor: add ECContext structure to encapsulate EC parameters
- Create ec_context.go with ECContext struct
- NewDefaultECContext() creates context with default 10+4 configuration
- Helper methods: CreateEncoder(), ToExt(), String()
- Foundation for cleaner function signatures
- No behavior change, still uses hardcoded 10+4
* refactor: update ec_encoder.go to use ECContext
- Add WriteEcFilesWithContext() and RebuildEcFilesWithContext() functions
- Keep old functions for backward compatibility (call new versions)
- Update all internal functions to accept ECContext parameter
- Use ctx.DataShards, ctx.ParityShards, ctx.TotalShards consistently
- Use ctx.CreateEncoder() instead of hardcoded reedsolomon.New()
- Use ctx.ToExt() for shard file extensions
- No behavior change, still uses default 10+4 configuration
* refactor: update ec_volume.go to use ECContext
- Add ECContext field to EcVolume struct
- Initialize ECContext with default configuration in NewEcVolume()
- Update LocateEcShardNeedleInterval() to use ECContext.DataShards
- Phase 1: Always uses default 10+4 configuration
- No behavior change
* refactor: add EC shard count fields to VolumeInfo protobuf
- Add data_shards_count field (field 8) to VolumeInfo message
- Add parity_shards_count field (field 9) to VolumeInfo message
- Fields are optional, 0 means use default (10+4)
- Backward compatible: fields added at end
- Phase 1: Foundation for future customization
* refactor: regenerate protobuf Go files with EC shard count fields
- Regenerated volume_server_pb/*.go with new EC fields
- DataShardsCount and ParityShardsCount accessors added to VolumeInfo
- No behavior change, fields not yet used
* refactor: update VolumeEcShardsGenerate to use ECContext
- Create ECContext with default configuration in VolumeEcShardsGenerate
- Use ecCtx.TotalShards and ecCtx.ToExt() in cleanup
- Call WriteEcFilesWithContext() instead of WriteEcFiles()
- Save EC configuration (DataShardsCount, ParityShardsCount) to VolumeInfo
- Log EC context being used
- Phase 1: Always uses default 10+4 configuration
- No behavior change
* fmt
* refactor: update ec_test.go to use ECContext
- Update TestEncodingDecoding to create and use ECContext
- Update validateFiles() to accept ECContext parameter
- Update removeGeneratedFiles() to use ctx.TotalShards and ctx.ToExt()
- Test passes with default 10+4 configuration
* refactor: use EcShardConfig message instead of separate fields
* optimize: pre-calculate row sizes in EC encoding loop
* refactor: replace TotalShards field with Total() method
- Remove TotalShards field from ECContext to avoid field drift
- Add Total() method that computes DataShards + ParityShards
- Update all references to use ctx.Total() instead of ctx.TotalShards
- Read EC config from VolumeInfo when loading EC volumes
- Read data shard count from .vif in VolumeEcShardsToVolume
- Use >= instead of > for exact boundary handling in encoding loops
* optimize: simplify VolumeEcShardsToVolume to use existing EC context
- Remove redundant CollectEcShards call
- Remove redundant .vif file loading
- Use v.ECContext.DataShards directly (already loaded by NewEcVolume)
- Slice tempShards instead of collecting again
* refactor: rename MaxShardId to MaxShardCount for clarity
- Change from MaxShardId=31 to MaxShardCount=32
- Eliminates confusing +1 arithmetic (MaxShardId+1)
- More intuitive: MaxShardCount directly represents the limit
fix: support custom EC ratios beyond 14 shards in VolumeEcShardsToVolume
- Add MaxShardId constant (31, since ShardBits is uint32)
- Use MaxShardId+1 (32) instead of TotalShardsCount (14) for tempShards buffer
- Prevents panic when slicing for volumes with >14 total shards
- Critical fix for custom EC configurations like 20+10
* fix: add validation for EC shard counts from VolumeInfo
- Validate DataShards/ParityShards are positive and within MaxShardCount
- Prevent zero or invalid values that could cause divide-by-zero
- Fallback to defaults if validation fails, with warning log
- VolumeEcShardsGenerate now preserves existing EC config when regenerating
- Critical safety fix for corrupted or legacy .vif files
* fix: RebuildEcFiles now loads EC config from .vif file
- Critical: RebuildEcFiles was always using default 10+4 config
- Now loads actual EC config from .vif file when rebuilding shards
- Validates config before use (positive shards, within MaxShardCount)
- Falls back to default if .vif missing or invalid
- Prevents data corruption when rebuilding custom EC volumes
* add: defensive validation for dataShards in VolumeEcShardsToVolume
- Validate dataShards > 0 and <= MaxShardCount before use
- Prevents panic from corrupted or uninitialized ECContext
- Returns clear error message instead of panic
- Defense-in-depth: validates even though upstream should catch issues
* fix: replace TotalShardsCount with MaxShardCount for custom EC ratio support
Critical fixes to support custom EC ratios > 14 shards:
disk_location_ec.go:
- validateEcVolume: Check shards 0-31 instead of 0-13 during validation
- removeEcVolumeFiles: Remove shards 0-31 instead of 0-13 during cleanup
ec_volume_info.go ShardBits methods:
- ShardIds(): Iterate up to MaxShardCount (32) instead of TotalShardsCount (14)
- ToUint32Slice(): Iterate up to MaxShardCount (32)
- IndexToShardId(): Iterate up to MaxShardCount (32)
- MinusParityShards(): Remove shards 10-31 instead of 10-13 (added note about Phase 2)
- Minus() shard size copy: Iterate up to MaxShardCount (32)
- resizeShardSizes(): Iterate up to MaxShardCount (32)
Without these changes:
- Custom EC ratios > 14 total shards would fail validation on startup
- Shards 14-31 would never be discovered or cleaned up
- ShardBits operations would miss shards >= 14
These changes are backward compatible - MaxShardCount (32) includes
the default TotalShardsCount (14), so existing 10+4 volumes work as before.
* fix: replace TotalShardsCount with MaxShardCount in critical data structures
Critical fixes for buffer allocations and loops that must support
custom EC ratios up to 32 shards:
Data Structures:
- store_ec.go:354: Buffer allocation for shard recovery (bufs array)
- topology_ec.go:14: EcShardLocations.Locations fixed array size
- command_ec_rebuild.go:268: EC shard map allocation
- command_ec_common.go:626: Shard-to-locations map allocation
Shard Discovery Loops:
- ec_task.go:378: Loop to find generated shard files
- ec_shard_management.go: All 8 loops that check/count EC shards
These changes are critical because:
1. Buffer allocations sized to 14 would cause index-out-of-bounds panics
when accessing shards 14-31
2. Fixed arrays sized to 14 would truncate shard location data
3. Loops limited to 0-13 would never discover/manage shards 14-31
Note: command_ec_encode.go:208 intentionally NOT changed - it creates
shard IDs to mount after encoding. In Phase 1 we always generate 14
shards, so this remains TotalShardsCount and will be made dynamic in
Phase 2 based on actual EC context.
Without these fixes, custom EC ratios > 14 total shards would cause:
- Runtime panics (array index out of bounds)
- Data loss (shards 14-31 never discovered/tracked)
- Incomplete shard management (missing shards not detected)
* refactor: move MaxShardCount constant to ec_encoder.go
Moved MaxShardCount from ec_volume_info.go to ec_encoder.go to group it
with other shard count constants (DataShardsCount, ParityShardsCount,
TotalShardsCount). This improves code organization and makes it easier
to understand the relationship between these constants.
Location: ec_encoder.go line 22, between TotalShardsCount and MinTotalDisks
* improve: add defensive programming and better error messages for EC
Code review improvements from CodeRabbit:
1. ShardBits Guardrails (ec_volume_info.go):
- AddShardId, RemoveShardId: Reject shard IDs >= MaxShardCount
- HasShardId: Return false for out-of-range shard IDs
- Prevents silent no-ops from bit shifts with invalid IDs
2. Future-Proof Regex (disk_location_ec.go):
- Updated regex from \.ec[0-9][0-9] to \.ec\d{2,3}
- Now matches .ec00 through .ec999 (currently .ec00-.ec31 used)
- Supports future increases to MaxShardCount beyond 99
3. Better Error Messages (volume_grpc_erasure_coding.go):
- Include valid range (1..32) in dataShards validation error
- Helps operators quickly identify the problem
4. Validation Before Save (volume_grpc_erasure_coding.go):
- Validate ECContext (DataShards > 0, ParityShards > 0, Total <= MaxShardCount)
- Log EC config being saved to .vif for debugging
- Prevents writing invalid configs to disk
These changes improve robustness and debuggability without changing
core functionality.
* fmt
* fix: critical bugs from code review + clean up comments
Critical bug fixes:
1. command_ec_rebuild.go: Fixed indentation causing compilation error
- Properly nested if/for blocks in registerEcNode
2. ec_shard_management.go: Fixed isComplete logic incorrectly using MaxShardCount
- Changed from MaxShardCount (32) back to TotalShardsCount (14)
- Default 10+4 volumes were being incorrectly reported as incomplete
- Missing shards 14-31 were being incorrectly reported as missing
- Fixed in 4 locations: volume completeness checks and getMissingShards
3. ec_volume_info.go: Fixed MinusParityShards removing too many shards
- Changed from MaxShardCount (32) back to TotalShardsCount (14)
- Was incorrectly removing shard IDs 10-31 instead of just 10-13
Comment cleanup:
- Removed Phase 1/Phase 2 references (development plan context)
- Replaced with clear statements about default 10+4 configuration
- SeaweedFS repo uses fixed 10+4 EC ratio, no phases needed
Root cause: Over-aggressive replacement of TotalShardsCount with MaxShardCount.
MaxShardCount (32) is the limit for buffer allocations and shard ID loops,
but TotalShardsCount (14) must be used for default EC configuration logic.
* fix: add defensive bounds checks and compute actual shard counts
Critical fixes from code review:
1. topology_ec.go: Add defensive bounds checks to AddShard/DeleteShard
- Prevent panic when shardId >= MaxShardCount (32)
- Return false instead of crashing on out-of-range shard IDs
2. command_ec_common.go: Fix doBalanceEcShardsAcrossRacks
- Was using hardcoded TotalShardsCount (14) for all volumes
- Now computes actual totalShardsForVolume from rackToShardCount
- Fixes incorrect rebalancing for volumes with custom EC ratios
- Example: 5+2=7 shards would incorrectly use 14 as average
These fixes improve robustness and prepare for future custom EC ratios
without changing current behavior for default 10+4 volumes.
Note: MinusParityShards and ec_task.go intentionally NOT changed for
seaweedfs repo - these will be enhanced in seaweed-enterprise repo
where custom EC ratio configuration is added.
* fmt
* style: make MaxShardCount type casting explicit in loops
Improved code clarity by explicitly casting MaxShardCount to the
appropriate type when used in loop comparisons:
- ShardId comparisons: Cast to ShardId(MaxShardCount)
- uint32 comparisons: Cast to uint32(MaxShardCount)
Changed in 5 locations:
- Minus() loop (line 90)
- ShardIds() loop (line 143)
- ToUint32Slice() loop (line 152)
- IndexToShardId() loop (line 219)
- resizeShardSizes() loop (line 248)
This makes the intent explicit and improves type safety readability.
No functional changes - purely a style improvement.
415 lines
14 KiB
Go
415 lines
14 KiB
Go
package storage
|
|
|
|
import (
|
|
"context"
|
|
"fmt"
|
|
"io"
|
|
"os"
|
|
"slices"
|
|
"sync"
|
|
"time"
|
|
|
|
"github.com/klauspost/reedsolomon"
|
|
|
|
"github.com/seaweedfs/seaweedfs/weed/glog"
|
|
"github.com/seaweedfs/seaweedfs/weed/operation"
|
|
"github.com/seaweedfs/seaweedfs/weed/pb"
|
|
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
|
|
"github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb"
|
|
"github.com/seaweedfs/seaweedfs/weed/stats"
|
|
"github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding"
|
|
"github.com/seaweedfs/seaweedfs/weed/storage/needle"
|
|
"github.com/seaweedfs/seaweedfs/weed/storage/types"
|
|
)
|
|
|
|
func (s *Store) CollectErasureCodingHeartbeat() *master_pb.Heartbeat {
|
|
var ecShardMessages []*master_pb.VolumeEcShardInformationMessage
|
|
collectionEcShardSize := make(map[string]int64)
|
|
for diskId, location := range s.Locations {
|
|
location.ecVolumesLock.RLock()
|
|
for _, ecShards := range location.ecVolumes {
|
|
ecShardMessages = append(ecShardMessages, ecShards.ToVolumeEcShardInformationMessage(uint32(diskId))...)
|
|
|
|
for _, ecShard := range ecShards.Shards {
|
|
collectionEcShardSize[ecShards.Collection] += ecShard.Size()
|
|
}
|
|
}
|
|
location.ecVolumesLock.RUnlock()
|
|
}
|
|
|
|
for col, size := range collectionEcShardSize {
|
|
stats.VolumeServerDiskSizeGauge.WithLabelValues(col, "ec").Set(float64(size))
|
|
}
|
|
|
|
return &master_pb.Heartbeat{
|
|
EcShards: ecShardMessages,
|
|
HasNoEcShards: len(ecShardMessages) == 0,
|
|
}
|
|
|
|
}
|
|
|
|
func (s *Store) MountEcShards(collection string, vid needle.VolumeId, shardId erasure_coding.ShardId) error {
|
|
for diskId, location := range s.Locations {
|
|
if ecVolume, err := location.LoadEcShard(collection, vid, shardId); err == nil {
|
|
glog.V(0).Infof("MountEcShards %d.%d on disk ID %d", vid, shardId, diskId)
|
|
|
|
var shardBits erasure_coding.ShardBits
|
|
|
|
s.NewEcShardsChan <- master_pb.VolumeEcShardInformationMessage{
|
|
Id: uint32(vid),
|
|
Collection: collection,
|
|
EcIndexBits: uint32(shardBits.AddShardId(shardId)),
|
|
DiskType: string(location.DiskType),
|
|
ExpireAtSec: ecVolume.ExpireAtSec,
|
|
DiskId: uint32(diskId),
|
|
}
|
|
return nil
|
|
} else if err == os.ErrNotExist {
|
|
continue
|
|
} else {
|
|
return fmt.Errorf("%s load ec shard %d.%d: %v", location.Directory, vid, shardId, err)
|
|
}
|
|
}
|
|
|
|
return fmt.Errorf("MountEcShards %d.%d not found on disk", vid, shardId)
|
|
}
|
|
|
|
func (s *Store) UnmountEcShards(vid needle.VolumeId, shardId erasure_coding.ShardId) error {
|
|
|
|
diskId, ecShard, found := s.findEcShard(vid, shardId)
|
|
if !found {
|
|
return nil
|
|
}
|
|
|
|
var shardBits erasure_coding.ShardBits
|
|
message := master_pb.VolumeEcShardInformationMessage{
|
|
Id: uint32(vid),
|
|
Collection: ecShard.Collection,
|
|
EcIndexBits: uint32(shardBits.AddShardId(shardId)),
|
|
DiskType: string(ecShard.DiskType),
|
|
DiskId: diskId,
|
|
}
|
|
|
|
location := s.Locations[diskId]
|
|
|
|
if deleted := location.UnloadEcShard(vid, shardId); deleted {
|
|
glog.V(0).Infof("UnmountEcShards %d.%d", vid, shardId)
|
|
s.DeletedEcShardsChan <- message
|
|
return nil
|
|
}
|
|
|
|
return fmt.Errorf("UnmountEcShards %d.%d not found on disk", vid, shardId)
|
|
}
|
|
|
|
func (s *Store) findEcShard(vid needle.VolumeId, shardId erasure_coding.ShardId) (diskId uint32, shard *erasure_coding.EcVolumeShard, found bool) {
|
|
for diskId, location := range s.Locations {
|
|
if v, found := location.FindEcShard(vid, shardId); found {
|
|
return uint32(diskId), v, found
|
|
}
|
|
}
|
|
return 0, nil, false
|
|
}
|
|
|
|
func (s *Store) FindEcVolume(vid needle.VolumeId) (*erasure_coding.EcVolume, bool) {
|
|
for _, location := range s.Locations {
|
|
if s, found := location.FindEcVolume(vid); found {
|
|
return s, true
|
|
}
|
|
}
|
|
return nil, false
|
|
}
|
|
|
|
// shardFiles is a list of shard files, which is used to return the shard locations
|
|
func (s *Store) CollectEcShards(vid needle.VolumeId, shardFileNames []string) (ecVolume *erasure_coding.EcVolume, found bool) {
|
|
for _, location := range s.Locations {
|
|
if s, foundShards := location.CollectEcShards(vid, shardFileNames); foundShards {
|
|
ecVolume = s
|
|
found = true
|
|
}
|
|
}
|
|
return
|
|
}
|
|
|
|
func (s *Store) DestroyEcVolume(vid needle.VolumeId) {
|
|
for _, location := range s.Locations {
|
|
location.DestroyEcVolume(vid)
|
|
}
|
|
}
|
|
|
|
func (s *Store) ReadEcShardNeedle(vid needle.VolumeId, n *needle.Needle, onReadSizeFn func(size types.Size)) (int, error) {
|
|
for _, location := range s.Locations {
|
|
if localEcVolume, found := location.FindEcVolume(vid); found {
|
|
|
|
offset, size, intervals, err := localEcVolume.LocateEcShardNeedle(n.Id, localEcVolume.Version)
|
|
if err != nil {
|
|
return 0, fmt.Errorf("locate in local ec volume: %w", err)
|
|
}
|
|
if size.IsDeleted() {
|
|
return 0, ErrorDeleted
|
|
}
|
|
|
|
if onReadSizeFn != nil {
|
|
onReadSizeFn(size)
|
|
}
|
|
|
|
glog.V(3).Infof("read ec volume %d offset %d size %d intervals:%+v", vid, offset.ToActualOffset(), size, intervals)
|
|
|
|
if len(intervals) > 1 {
|
|
glog.V(3).Infof("ReadEcShardNeedle needle id %s intervals:%+v", n.String(), intervals)
|
|
}
|
|
bytes, isDeleted, err := s.readEcShardIntervals(vid, n.Id, localEcVolume, intervals)
|
|
if err != nil {
|
|
return 0, fmt.Errorf("ReadEcShardIntervals: %w", err)
|
|
}
|
|
if isDeleted {
|
|
return 0, ErrorDeleted
|
|
}
|
|
|
|
err = n.ReadBytes(bytes, offset.ToActualOffset(), size, localEcVolume.Version)
|
|
if err != nil {
|
|
return 0, fmt.Errorf("readbytes: %w", err)
|
|
}
|
|
|
|
return len(bytes), nil
|
|
}
|
|
}
|
|
return 0, fmt.Errorf("ec shard %d not found", vid)
|
|
}
|
|
|
|
func (s *Store) readEcShardIntervals(vid needle.VolumeId, needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, intervals []erasure_coding.Interval) (data []byte, is_deleted bool, err error) {
|
|
|
|
if err = s.cachedLookupEcShardLocations(ecVolume); err != nil {
|
|
return nil, false, fmt.Errorf("failed to locate shard via master grpc %s: %v", s.MasterAddress, err)
|
|
}
|
|
|
|
for i, interval := range intervals {
|
|
if d, isDeleted, e := s.readOneEcShardInterval(needleId, ecVolume, interval); e != nil {
|
|
return nil, isDeleted, e
|
|
} else {
|
|
if isDeleted {
|
|
is_deleted = true
|
|
}
|
|
if i == 0 {
|
|
data = d
|
|
} else {
|
|
data = append(data, d...)
|
|
}
|
|
}
|
|
}
|
|
return
|
|
}
|
|
|
|
func (s *Store) readOneEcShardInterval(needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, interval erasure_coding.Interval) (data []byte, is_deleted bool, err error) {
|
|
shardId, actualOffset := interval.ToShardIdAndOffset(erasure_coding.ErasureCodingLargeBlockSize, erasure_coding.ErasureCodingSmallBlockSize)
|
|
data = make([]byte, interval.Size)
|
|
if shard, found := ecVolume.FindEcVolumeShard(shardId); found {
|
|
var readSize int
|
|
if readSize, err = shard.ReadAt(data, actualOffset); err != nil {
|
|
if readSize != int(interval.Size) {
|
|
glog.V(0).Infof("read local ec shard %d.%d offset %d: %v", ecVolume.VolumeId, shardId, actualOffset, err)
|
|
return
|
|
}
|
|
}
|
|
} else {
|
|
ecVolume.ShardLocationsLock.RLock()
|
|
sourceDataNodes, hasShardIdLocation := ecVolume.ShardLocations[shardId]
|
|
ecVolume.ShardLocationsLock.RUnlock()
|
|
|
|
// try reading directly
|
|
if hasShardIdLocation {
|
|
_, is_deleted, err = s.readRemoteEcShardInterval(sourceDataNodes, needleId, ecVolume.VolumeId, shardId, data, actualOffset)
|
|
if err == nil {
|
|
return
|
|
}
|
|
glog.V(0).Infof("clearing ec shard %d.%d locations: %v", ecVolume.VolumeId, shardId, err)
|
|
}
|
|
|
|
// try reading by recovering from other shards
|
|
_, is_deleted, err = s.recoverOneRemoteEcShardInterval(needleId, ecVolume, shardId, data, actualOffset)
|
|
if err == nil {
|
|
return
|
|
}
|
|
glog.V(0).Infof("recover ec shard %d.%d : %v", ecVolume.VolumeId, shardId, err)
|
|
}
|
|
return
|
|
}
|
|
|
|
func forgetShardId(ecVolume *erasure_coding.EcVolume, shardId erasure_coding.ShardId) {
|
|
// failed to access the source data nodes, clear it up
|
|
ecVolume.ShardLocationsLock.Lock()
|
|
delete(ecVolume.ShardLocations, shardId)
|
|
ecVolume.ShardLocationsLock.Unlock()
|
|
}
|
|
|
|
func (s *Store) cachedLookupEcShardLocations(ecVolume *erasure_coding.EcVolume) (err error) {
|
|
|
|
shardCount := len(ecVolume.ShardLocations)
|
|
if shardCount < erasure_coding.DataShardsCount &&
|
|
ecVolume.ShardLocationsRefreshTime.Add(11*time.Second).After(time.Now()) ||
|
|
shardCount == erasure_coding.TotalShardsCount &&
|
|
ecVolume.ShardLocationsRefreshTime.Add(37*time.Minute).After(time.Now()) ||
|
|
shardCount >= erasure_coding.DataShardsCount &&
|
|
ecVolume.ShardLocationsRefreshTime.Add(7*time.Minute).After(time.Now()) {
|
|
// still fresh
|
|
return nil
|
|
}
|
|
|
|
glog.V(3).Infof("lookup and cache ec volume %d locations", ecVolume.VolumeId)
|
|
|
|
err = operation.WithMasterServerClient(false, s.MasterAddress, s.grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
|
|
req := &master_pb.LookupEcVolumeRequest{
|
|
VolumeId: uint32(ecVolume.VolumeId),
|
|
}
|
|
resp, err := masterClient.LookupEcVolume(context.Background(), req)
|
|
if err != nil {
|
|
return fmt.Errorf("lookup ec volume %d: %v", ecVolume.VolumeId, err)
|
|
}
|
|
if len(resp.ShardIdLocations) < erasure_coding.DataShardsCount {
|
|
return fmt.Errorf("only %d shards found but %d required", len(resp.ShardIdLocations), erasure_coding.DataShardsCount)
|
|
}
|
|
|
|
ecVolume.ShardLocationsLock.Lock()
|
|
for _, shardIdLocations := range resp.ShardIdLocations {
|
|
shardId := erasure_coding.ShardId(shardIdLocations.ShardId)
|
|
delete(ecVolume.ShardLocations, shardId)
|
|
for _, loc := range shardIdLocations.Locations {
|
|
ecVolume.ShardLocations[shardId] = append(ecVolume.ShardLocations[shardId], pb.NewServerAddressFromLocation(loc))
|
|
}
|
|
}
|
|
ecVolume.ShardLocationsRefreshTime = time.Now()
|
|
ecVolume.ShardLocationsLock.Unlock()
|
|
|
|
return nil
|
|
})
|
|
return
|
|
}
|
|
|
|
func (s *Store) readRemoteEcShardInterval(sourceDataNodes []pb.ServerAddress, needleId types.NeedleId, vid needle.VolumeId, shardId erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
|
|
|
|
if len(sourceDataNodes) == 0 {
|
|
return 0, false, fmt.Errorf("failed to find ec shard %d.%d", vid, shardId)
|
|
}
|
|
|
|
for _, sourceDataNode := range sourceDataNodes {
|
|
glog.V(3).Infof("read remote ec shard %d.%d from %s", vid, shardId, sourceDataNode)
|
|
n, is_deleted, err = s.doReadRemoteEcShardInterval(sourceDataNode, needleId, vid, shardId, buf, offset)
|
|
if err == nil {
|
|
return
|
|
}
|
|
glog.V(1).Infof("read remote ec shard %d.%d from %s: %v", vid, shardId, sourceDataNode, err)
|
|
}
|
|
|
|
return
|
|
}
|
|
|
|
func (s *Store) doReadRemoteEcShardInterval(sourceDataNode pb.ServerAddress, needleId types.NeedleId, vid needle.VolumeId, shardId erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
|
|
|
|
err = operation.WithVolumeServerClient(false, sourceDataNode, s.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
|
|
|
|
// copy data slice
|
|
shardReadClient, err := client.VolumeEcShardRead(context.Background(), &volume_server_pb.VolumeEcShardReadRequest{
|
|
VolumeId: uint32(vid),
|
|
ShardId: uint32(shardId),
|
|
Offset: offset,
|
|
Size: int64(len(buf)),
|
|
FileKey: uint64(needleId),
|
|
})
|
|
if err != nil {
|
|
return fmt.Errorf("failed to start reading ec shard %d.%d from %s: %v", vid, shardId, sourceDataNode, err)
|
|
}
|
|
|
|
for {
|
|
resp, receiveErr := shardReadClient.Recv()
|
|
if receiveErr == io.EOF {
|
|
break
|
|
}
|
|
if receiveErr != nil {
|
|
return fmt.Errorf("receiving ec shard %d.%d from %s: %v", vid, shardId, sourceDataNode, receiveErr)
|
|
}
|
|
if resp.IsDeleted {
|
|
is_deleted = true
|
|
}
|
|
copy(buf[n:n+len(resp.Data)], resp.Data)
|
|
n += len(resp.Data)
|
|
}
|
|
|
|
return nil
|
|
})
|
|
if err != nil {
|
|
return 0, is_deleted, fmt.Errorf("read ec shard %d.%d from %s: %v", vid, shardId, sourceDataNode, err)
|
|
}
|
|
|
|
return
|
|
}
|
|
|
|
func (s *Store) recoverOneRemoteEcShardInterval(needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, shardIdToRecover erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
|
|
glog.V(3).Infof("recover ec shard %d.%d from other locations", ecVolume.VolumeId, shardIdToRecover)
|
|
|
|
enc, err := reedsolomon.New(erasure_coding.DataShardsCount, erasure_coding.ParityShardsCount)
|
|
if err != nil {
|
|
return 0, false, fmt.Errorf("failed to create encoder: %w", err)
|
|
}
|
|
|
|
// Use MaxShardCount to support custom EC ratios up to 32 shards
|
|
bufs := make([][]byte, erasure_coding.MaxShardCount)
|
|
|
|
var wg sync.WaitGroup
|
|
ecVolume.ShardLocationsLock.RLock()
|
|
for shardId, locations := range ecVolume.ShardLocations {
|
|
|
|
// skip current shard or empty shard
|
|
if shardId == shardIdToRecover {
|
|
continue
|
|
}
|
|
if len(locations) == 0 {
|
|
glog.V(3).Infof("readRemoteEcShardInterval missing %d.%d from %+v", ecVolume.VolumeId, shardId, locations)
|
|
continue
|
|
}
|
|
|
|
// read from remote locations
|
|
wg.Add(1)
|
|
go func(shardId erasure_coding.ShardId, locations []pb.ServerAddress) {
|
|
defer wg.Done()
|
|
data := make([]byte, len(buf))
|
|
nRead, isDeleted, readErr := s.readRemoteEcShardInterval(locations, needleId, ecVolume.VolumeId, shardId, data, offset)
|
|
if readErr != nil {
|
|
glog.V(3).Infof("recover: readRemoteEcShardInterval %d.%d %d bytes from %+v: %v", ecVolume.VolumeId, shardId, nRead, locations, readErr)
|
|
forgetShardId(ecVolume, shardId)
|
|
}
|
|
if isDeleted {
|
|
is_deleted = true
|
|
}
|
|
if nRead == len(buf) {
|
|
bufs[shardId] = data
|
|
}
|
|
}(shardId, locations)
|
|
}
|
|
ecVolume.ShardLocationsLock.RUnlock()
|
|
|
|
wg.Wait()
|
|
|
|
if err = enc.ReconstructData(bufs); err != nil {
|
|
glog.V(3).Infof("recovered ec shard %d.%d failed: %v", ecVolume.VolumeId, shardIdToRecover, err)
|
|
return 0, false, err
|
|
}
|
|
glog.V(4).Infof("recovered ec shard %d.%d from other locations", ecVolume.VolumeId, shardIdToRecover)
|
|
|
|
copy(buf, bufs[shardIdToRecover])
|
|
|
|
return len(buf), is_deleted, nil
|
|
}
|
|
|
|
func (s *Store) EcVolumes() (ecVolumes []*erasure_coding.EcVolume) {
|
|
for _, location := range s.Locations {
|
|
location.ecVolumesLock.RLock()
|
|
for _, v := range location.ecVolumes {
|
|
ecVolumes = append(ecVolumes, v)
|
|
}
|
|
location.ecVolumesLock.RUnlock()
|
|
}
|
|
slices.SortFunc(ecVolumes, func(a, b *erasure_coding.EcVolume) int {
|
|
return int(a.VolumeId) - int(b.VolumeId)
|
|
})
|
|
return ecVolumes
|
|
}
|