2025-07-30 12:38:03 -07:00
package erasure_coding
import (
"fmt"
"strings"
"time"
2025-08-01 11:18:32 -07:00
"github.com/seaweedfs/seaweedfs/weed/admin/topology"
2025-07-30 12:38:03 -07:00
"github.com/seaweedfs/seaweedfs/weed/glog"
2025-08-01 11:18:32 -07:00
"github.com/seaweedfs/seaweedfs/weed/pb/worker_pb"
"github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding"
2025-07-30 12:38:03 -07:00
"github.com/seaweedfs/seaweedfs/weed/worker/tasks/base"
"github.com/seaweedfs/seaweedfs/weed/worker/types"
)
// Detection implements the detection logic for erasure coding tasks
func Detection ( metrics [ ] * types . VolumeHealthMetrics , clusterInfo * types . ClusterInfo , config base . TaskConfig ) ( [ ] * types . TaskDetectionResult , error ) {
if ! config . IsEnabled ( ) {
return nil , nil
}
ecConfig := config . ( * Config )
var results [ ] * types . TaskDetectionResult
now := time . Now ( )
quietThreshold := time . Duration ( ecConfig . QuietForSeconds ) * time . Second
minSizeBytes := uint64 ( ecConfig . MinSizeMB ) * 1024 * 1024 // Configurable minimum
debugCount := 0
skippedAlreadyEC := 0
skippedTooSmall := 0
skippedCollectionFilter := 0
skippedQuietTime := 0
skippedFullness := 0
for _ , metric := range metrics {
// Skip if already EC volume
if metric . IsECVolume {
skippedAlreadyEC ++
continue
}
// Check minimum size requirement
if metric . Size < minSizeBytes {
skippedTooSmall ++
continue
}
// Check collection filter if specified
if ecConfig . CollectionFilter != "" {
// Parse comma-separated collections
allowedCollections := make ( map [ string ] bool )
for _ , collection := range strings . Split ( ecConfig . CollectionFilter , "," ) {
allowedCollections [ strings . TrimSpace ( collection ) ] = true
}
// Skip if volume's collection is not in the allowed list
if ! allowedCollections [ metric . Collection ] {
skippedCollectionFilter ++
continue
}
}
// Check quiet duration and fullness criteria
if metric . Age >= quietThreshold && metric . FullnessRatio >= ecConfig . FullnessRatio {
2025-08-09 21:47:29 -07:00
glog . Infof ( "EC Detection: Volume %d meets all criteria, attempting to create task" , metric . VolumeID )
2025-08-03 01:35:38 -07:00
// Generate task ID for ActiveTopology integration
taskID := fmt . Sprintf ( "ec_vol_%d_%d" , metric . VolumeID , now . Unix ( ) )
2025-07-30 12:38:03 -07:00
result := & types . TaskDetectionResult {
2025-08-03 01:35:38 -07:00
TaskID : taskID , // Link to ActiveTopology pending task
2025-07-30 12:38:03 -07:00
TaskType : types . TaskTypeErasureCoding ,
VolumeID : metric . VolumeID ,
Server : metric . Server ,
Collection : metric . Collection ,
Priority : types . TaskPriorityLow , // EC is not urgent
2025-07-30 16:43:14 -07:00
Reason : fmt . Sprintf ( "Volume meets EC criteria: quiet for %.1fs (>%ds), fullness=%.1f%% (>%.1f%%), size=%.1fMB (>%dMB)" ,
2025-07-30 12:38:03 -07:00
metric . Age . Seconds ( ) , ecConfig . QuietForSeconds , metric . FullnessRatio * 100 , ecConfig . FullnessRatio * 100 ,
2025-07-30 16:43:14 -07:00
float64 ( metric . Size ) / ( 1024 * 1024 ) , ecConfig . MinSizeMB ) ,
2025-07-30 12:38:03 -07:00
ScheduleAt : now ,
}
2025-08-01 11:18:32 -07:00
// Plan EC destinations if ActiveTopology is available
if clusterInfo . ActiveTopology != nil {
2025-08-09 21:47:29 -07:00
glog . Infof ( "EC Detection: ActiveTopology available, planning destinations for volume %d" , metric . VolumeID )
2025-08-01 11:18:32 -07:00
multiPlan , err := planECDestinations ( clusterInfo . ActiveTopology , metric , ecConfig )
if err != nil {
glog . Warningf ( "Failed to plan EC destinations for volume %d: %v" , metric . VolumeID , err )
continue // Skip this volume if destination planning fails
}
2025-08-09 21:47:29 -07:00
glog . Infof ( "EC Detection: Successfully planned %d destinations for volume %d" , len ( multiPlan . Plans ) , metric . VolumeID )
2025-08-01 11:18:32 -07:00
2025-08-03 01:35:38 -07:00
// Calculate expected shard size for EC operation
// Each data shard will be approximately volumeSize / dataShards
expectedShardSize := uint64 ( metric . Size ) / uint64 ( erasure_coding . DataShardsCount )
// Add pending EC shard task to ActiveTopology for capacity management
// Extract shard destinations from multiPlan
var shardDestinations [ ] string
var shardDiskIDs [ ] uint32
for _ , plan := range multiPlan . Plans {
shardDestinations = append ( shardDestinations , plan . TargetNode )
shardDiskIDs = append ( shardDiskIDs , plan . TargetDisk )
}
// Find all volume replica locations (server + disk) from topology
2025-08-09 21:47:29 -07:00
glog . Infof ( "EC Detection: Looking for replica locations for volume %d" , metric . VolumeID )
2025-08-03 01:35:38 -07:00
replicaLocations := findVolumeReplicaLocations ( clusterInfo . ActiveTopology , metric . VolumeID , metric . Collection )
if len ( replicaLocations ) == 0 {
glog . Warningf ( "No replica locations found for volume %d, skipping EC" , metric . VolumeID )
continue
}
2025-08-09 21:47:29 -07:00
glog . Infof ( "EC Detection: Found %d replica locations for volume %d" , len ( replicaLocations ) , metric . VolumeID )
2025-08-03 01:35:38 -07:00
// Find existing EC shards from previous failed attempts
existingECShards := findExistingECShards ( clusterInfo . ActiveTopology , metric . VolumeID , metric . Collection )
// Combine volume replicas and existing EC shards for cleanup
2025-08-09 21:47:29 -07:00
var sources [ ] topology . TaskSourceSpec
2025-08-03 01:35:38 -07:00
// Add volume replicas (will free volume slots)
for _ , replica := range replicaLocations {
2025-08-09 21:47:29 -07:00
sources = append ( sources , topology . TaskSourceSpec {
2025-08-03 01:35:38 -07:00
ServerID : replica . ServerID ,
DiskID : replica . DiskID ,
2025-08-09 21:47:29 -07:00
DataCenter : replica . DataCenter ,
Rack : replica . Rack ,
2025-08-03 01:35:38 -07:00
CleanupType : topology . CleanupVolumeReplica ,
} )
}
// Add existing EC shards (will free shard slots)
duplicateCheck := make ( map [ string ] bool )
for _ , replica := range replicaLocations {
key := fmt . Sprintf ( "%s:%d" , replica . ServerID , replica . DiskID )
duplicateCheck [ key ] = true
}
for _ , shard := range existingECShards {
key := fmt . Sprintf ( "%s:%d" , shard . ServerID , shard . DiskID )
if ! duplicateCheck [ key ] { // Avoid duplicates if EC shards are on same disk as volume replicas
2025-08-09 21:47:29 -07:00
sources = append ( sources , topology . TaskSourceSpec {
2025-08-03 01:35:38 -07:00
ServerID : shard . ServerID ,
DiskID : shard . DiskID ,
2025-08-09 21:47:29 -07:00
DataCenter : shard . DataCenter ,
Rack : shard . Rack ,
2025-08-03 01:35:38 -07:00
CleanupType : topology . CleanupECShards ,
} )
duplicateCheck [ key ] = true
}
}
glog . V ( 2 ) . Infof ( "Found %d volume replicas and %d existing EC shards for volume %d (total %d cleanup sources)" ,
2025-08-09 21:47:29 -07:00
len ( replicaLocations ) , len ( existingECShards ) , metric . VolumeID , len ( sources ) )
2025-08-03 01:35:38 -07:00
// Convert shard destinations to TaskDestinationSpec
destinations := make ( [ ] topology . TaskDestinationSpec , len ( shardDestinations ) )
shardImpact := topology . CalculateECShardStorageImpact ( 1 , int64 ( expectedShardSize ) ) // 1 shard per destination
shardSize := int64 ( expectedShardSize )
for i , dest := range shardDestinations {
destinations [ i ] = topology . TaskDestinationSpec {
ServerID : dest ,
DiskID : shardDiskIDs [ i ] ,
StorageImpact : & shardImpact ,
EstimatedSize : & shardSize ,
}
}
err = clusterInfo . ActiveTopology . AddPendingTask ( topology . TaskSpec {
TaskID : taskID ,
TaskType : topology . TaskTypeErasureCoding ,
VolumeID : metric . VolumeID ,
VolumeSize : int64 ( metric . Size ) ,
Sources : sources ,
Destinations : destinations ,
} )
if err != nil {
glog . Warningf ( "Failed to add pending EC shard task to ActiveTopology for volume %d: %v" , metric . VolumeID , err )
continue // Skip this volume if topology task addition fails
}
glog . V ( 2 ) . Infof ( "Added pending EC shard task %s to ActiveTopology for volume %d with %d cleanup sources and %d shard destinations" ,
2025-08-09 21:47:29 -07:00
taskID , metric . VolumeID , len ( sources ) , len ( multiPlan . Plans ) )
2025-08-01 11:18:32 -07:00
2025-08-09 21:47:29 -07:00
// Create unified sources and targets for EC task
2025-08-01 11:18:32 -07:00
result . TypedParams = & worker_pb . TaskParams {
2025-08-03 01:35:38 -07:00
TaskId : taskID , // Link to ActiveTopology pending task
2025-08-01 11:18:32 -07:00
VolumeId : metric . VolumeID ,
Collection : metric . Collection ,
VolumeSize : metric . Size , // Store original volume size for tracking changes
2025-08-09 21:47:29 -07:00
// Unified sources - all sources that will be processed/cleaned up
Sources : convertTaskSourcesToProtobuf ( sources , metric . VolumeID ) ,
// Unified targets - all EC shard destinations
Targets : createECTargets ( multiPlan ) ,
2025-08-01 11:18:32 -07:00
TaskParams : & worker_pb . TaskParams_ErasureCodingParams {
ErasureCodingParams : createECTaskParams ( multiPlan ) ,
} ,
}
glog . V ( 1 ) . Infof ( "Planned EC destinations for volume %d: %d shards across %d racks, %d DCs" ,
metric . VolumeID , len ( multiPlan . Plans ) , multiPlan . SuccessfulRack , multiPlan . SuccessfulDCs )
} else {
glog . Warningf ( "No ActiveTopology available for destination planning in EC detection" )
continue // Skip this volume if no topology available
}
2025-08-09 21:47:29 -07:00
glog . Infof ( "EC Detection: Successfully created EC task for volume %d, adding to results" , metric . VolumeID )
2025-07-30 12:38:03 -07:00
results = append ( results , result )
} else {
// Count debug reasons
if debugCount < 5 { // Limit to avoid spam
if metric . Age < quietThreshold {
skippedQuietTime ++
}
if metric . FullnessRatio < ecConfig . FullnessRatio {
skippedFullness ++
}
}
debugCount ++
}
}
// Log debug summary if no tasks were created
if len ( results ) == 0 && len ( metrics ) > 0 {
totalVolumes := len ( metrics )
glog . V ( 1 ) . Infof ( "EC detection: No tasks created for %d volumes (skipped: %d already EC, %d too small, %d filtered, %d not quiet, %d not full)" ,
totalVolumes , skippedAlreadyEC , skippedTooSmall , skippedCollectionFilter , skippedQuietTime , skippedFullness )
// Show details for first few volumes
for i , metric := range metrics {
if i >= 3 || metric . IsECVolume { // Limit to first 3 non-EC volumes
continue
}
sizeMB := float64 ( metric . Size ) / ( 1024 * 1024 )
glog . Infof ( "ERASURE CODING: Volume %d: size=%.1fMB (need ≥%dMB), age=%s (need ≥%s), fullness=%.1f%% (need ≥%.1f%%)" ,
metric . VolumeID , sizeMB , ecConfig . MinSizeMB , metric . Age . Truncate ( time . Minute ) , quietThreshold . Truncate ( time . Minute ) ,
metric . FullnessRatio * 100 , ecConfig . FullnessRatio * 100 )
}
}
return results , nil
}
2025-08-01 11:18:32 -07:00
// planECDestinations plans the destinations for erasure coding operation
// This function implements EC destination planning logic directly in the detection phase
func planECDestinations ( activeTopology * topology . ActiveTopology , metric * types . VolumeHealthMetrics , ecConfig * Config ) ( * topology . MultiDestinationPlan , error ) {
2025-08-03 01:35:38 -07:00
// Calculate expected shard size for EC operation
expectedShardSize := uint64 ( metric . Size ) / uint64 ( erasure_coding . DataShardsCount )
2025-08-01 11:18:32 -07:00
// Get source node information from topology
var sourceRack , sourceDC string
2025-07-30 12:38:03 -07:00
2025-08-01 11:18:32 -07:00
// Extract rack and DC from topology info
topologyInfo := activeTopology . GetTopologyInfo ( )
if topologyInfo != nil {
for _ , dc := range topologyInfo . DataCenterInfos {
for _ , rack := range dc . RackInfos {
for _ , dataNodeInfo := range rack . DataNodeInfos {
if dataNodeInfo . Id == metric . Server {
sourceDC = dc . Id
sourceRack = rack . Id
break
}
}
if sourceRack != "" {
break
}
}
if sourceDC != "" {
break
}
}
}
2025-08-03 01:35:38 -07:00
// Get available disks for EC placement with effective capacity consideration (includes pending tasks)
// For EC, we typically need 1 volume slot per shard, so use minimum capacity of 1
// For EC, we need at least 1 available volume slot on a disk to consider it for placement.
2025-08-09 21:47:29 -07:00
// Note: We don't exclude the source server since the original volume will be deleted after EC conversion
availableDisks := activeTopology . GetDisksWithEffectiveCapacity ( topology . TaskTypeErasureCoding , "" , 1 )
2025-08-02 02:16:49 -07:00
if len ( availableDisks ) < erasure_coding . MinTotalDisks {
2025-08-03 01:35:38 -07:00
return nil , fmt . Errorf ( "insufficient disks for EC placement: need %d, have %d (considering pending/active tasks)" , erasure_coding . MinTotalDisks , len ( availableDisks ) )
2025-08-01 11:18:32 -07:00
}
// Select best disks for EC placement with rack/DC diversity
selectedDisks := selectBestECDestinations ( availableDisks , sourceRack , sourceDC , erasure_coding . TotalShardsCount )
2025-08-02 02:16:49 -07:00
if len ( selectedDisks ) < erasure_coding . MinTotalDisks {
return nil , fmt . Errorf ( "found %d disks, but could not find %d suitable destinations for EC placement" , len ( selectedDisks ) , erasure_coding . MinTotalDisks )
2025-08-01 11:18:32 -07:00
}
var plans [ ] * topology . DestinationPlan
rackCount := make ( map [ string ] int )
dcCount := make ( map [ string ] int )
for _ , disk := range selectedDisks {
plan := & topology . DestinationPlan {
TargetNode : disk . NodeID ,
TargetDisk : disk . DiskID ,
TargetRack : disk . Rack ,
TargetDC : disk . DataCenter ,
2025-08-03 01:35:38 -07:00
ExpectedSize : expectedShardSize , // Set calculated EC shard size
2025-08-01 11:18:32 -07:00
PlacementScore : calculateECScore ( disk , sourceRack , sourceDC ) ,
}
plans = append ( plans , plan )
// Count rack and DC diversity
rackKey := fmt . Sprintf ( "%s:%s" , disk . DataCenter , disk . Rack )
rackCount [ rackKey ] ++
dcCount [ disk . DataCenter ] ++
}
2025-08-03 01:35:38 -07:00
// Log capacity utilization information using ActiveTopology's encapsulated logic
totalEffectiveCapacity := int64 ( 0 )
for _ , plan := range plans {
effectiveCapacity := activeTopology . GetEffectiveAvailableCapacity ( plan . TargetNode , plan . TargetDisk )
totalEffectiveCapacity += effectiveCapacity
}
glog . V ( 1 ) . Infof ( "Planned EC destinations for volume %d (size=%d bytes): expected shard size=%d bytes, %d shards across %d racks, %d DCs, total effective capacity=%d slots" ,
metric . VolumeID , metric . Size , expectedShardSize , len ( plans ) , len ( rackCount ) , len ( dcCount ) , totalEffectiveCapacity )
// Log storage impact for EC task (source only - EC has multiple targets handled individually)
sourceChange , _ := topology . CalculateTaskStorageImpact ( topology . TaskTypeErasureCoding , int64 ( metric . Size ) )
glog . V ( 2 ) . Infof ( "EC task capacity management: source_reserves_with_zero_impact={VolumeSlots:%d, ShardSlots:%d}, %d_targets_will_receive_shards, estimated_size=%d" ,
sourceChange . VolumeSlots , sourceChange . ShardSlots , len ( plans ) , metric . Size )
glog . V ( 2 ) . Infof ( "EC source reserves capacity but with zero StorageSlotChange impact" )
2025-08-01 11:18:32 -07:00
return & topology . MultiDestinationPlan {
Plans : plans ,
TotalShards : len ( plans ) ,
SuccessfulRack : len ( rackCount ) ,
SuccessfulDCs : len ( dcCount ) ,
} , nil
}
2025-08-09 21:47:29 -07:00
// createECTargets creates unified TaskTarget structures from the multi-destination plan
// with proper shard ID assignment during planning phase
func createECTargets ( multiPlan * topology . MultiDestinationPlan ) [ ] * worker_pb . TaskTarget {
var targets [ ] * worker_pb . TaskTarget
numTargets := len ( multiPlan . Plans )
// Create shard assignment arrays for each target (round-robin distribution)
targetShards := make ( [ ] [ ] uint32 , numTargets )
for i := range targetShards {
targetShards [ i ] = make ( [ ] uint32 , 0 )
}
// Distribute shards in round-robin fashion to spread both data and parity shards
// This ensures each target gets a mix of data shards (0-9) and parity shards (10-13)
for shardId := uint32 ( 0 ) ; shardId < uint32 ( erasure_coding . TotalShardsCount ) ; shardId ++ {
targetIndex := int ( shardId ) % numTargets
targetShards [ targetIndex ] = append ( targetShards [ targetIndex ] , shardId )
}
// Create targets with assigned shard IDs
for i , plan := range multiPlan . Plans {
target := & worker_pb . TaskTarget {
Node : plan . TargetNode ,
DiskId : plan . TargetDisk ,
Rack : plan . TargetRack ,
DataCenter : plan . TargetDC ,
ShardIds : targetShards [ i ] , // Round-robin assigned shards
EstimatedSize : plan . ExpectedSize ,
}
targets = append ( targets , target )
// Log shard assignment with data/parity classification
dataShards := make ( [ ] uint32 , 0 )
parityShards := make ( [ ] uint32 , 0 )
for _ , shardId := range targetShards [ i ] {
if shardId < uint32 ( erasure_coding . DataShardsCount ) {
dataShards = append ( dataShards , shardId )
} else {
parityShards = append ( parityShards , shardId )
}
2025-08-01 11:18:32 -07:00
}
2025-08-09 21:47:29 -07:00
glog . V ( 2 ) . Infof ( "EC planning: target %s assigned shards %v (data: %v, parity: %v)" ,
plan . TargetNode , targetShards [ i ] , dataShards , parityShards )
2025-08-01 11:18:32 -07:00
}
2025-08-09 21:47:29 -07:00
glog . V ( 1 ) . Infof ( "EC planning: distributed %d shards across %d targets using round-robin (data shards 0-%d, parity shards %d-%d)" ,
erasure_coding . TotalShardsCount , numTargets ,
erasure_coding . DataShardsCount - 1 , erasure_coding . DataShardsCount , erasure_coding . TotalShardsCount - 1 )
return targets
}
// convertTaskSourcesToProtobuf converts topology.TaskSourceSpec to worker_pb.TaskSource
func convertTaskSourcesToProtobuf ( sources [ ] topology . TaskSourceSpec , volumeID uint32 ) [ ] * worker_pb . TaskSource {
var protobufSources [ ] * worker_pb . TaskSource
for _ , source := range sources {
pbSource := & worker_pb . TaskSource {
Node : source . ServerID ,
DiskId : source . DiskID ,
DataCenter : source . DataCenter ,
Rack : source . Rack ,
}
// Convert storage impact to estimated size
if source . EstimatedSize != nil {
pbSource . EstimatedSize = uint64 ( * source . EstimatedSize )
}
// Set appropriate volume ID or shard IDs based on cleanup type
switch source . CleanupType {
case topology . CleanupVolumeReplica :
// This is a volume replica, use the actual volume ID
pbSource . VolumeId = volumeID
case topology . CleanupECShards :
// This is EC shards, also use the volume ID for consistency
pbSource . VolumeId = volumeID
// Note: ShardIds would need to be passed separately if we need specific shard info
}
protobufSources = append ( protobufSources , pbSource )
2025-08-01 11:18:32 -07:00
}
2025-08-09 21:47:29 -07:00
return protobufSources
}
// createECTaskParams creates clean EC task parameters (destinations now in unified targets)
func createECTaskParams ( multiPlan * topology . MultiDestinationPlan ) * worker_pb . ErasureCodingTaskParams {
2025-08-01 11:18:32 -07:00
return & worker_pb . ErasureCodingTaskParams {
2025-08-09 21:47:29 -07:00
DataShards : erasure_coding . DataShardsCount , // Standard data shards
ParityShards : erasure_coding . ParityShardsCount , // Standard parity shards
2025-08-01 11:18:32 -07:00
}
}
// selectBestECDestinations selects multiple disks for EC shard placement with diversity
func selectBestECDestinations ( disks [ ] * topology . DiskInfo , sourceRack , sourceDC string , shardsNeeded int ) [ ] * topology . DiskInfo {
if len ( disks ) == 0 {
return nil
}
// Group disks by rack and DC for diversity
rackGroups := make ( map [ string ] [ ] * topology . DiskInfo )
for _ , disk := range disks {
rackKey := fmt . Sprintf ( "%s:%s" , disk . DataCenter , disk . Rack )
rackGroups [ rackKey ] = append ( rackGroups [ rackKey ] , disk )
}
var selected [ ] * topology . DiskInfo
usedRacks := make ( map [ string ] bool )
// First pass: select one disk from each rack for maximum diversity
for rackKey , rackDisks := range rackGroups {
if len ( selected ) >= shardsNeeded {
break
}
// Select best disk from this rack
bestDisk := selectBestFromRack ( rackDisks , sourceRack , sourceDC )
if bestDisk != nil {
selected = append ( selected , bestDisk )
usedRacks [ rackKey ] = true
}
}
// Second pass: if we need more disks, select from racks we've already used
if len ( selected ) < shardsNeeded {
for _ , disk := range disks {
if len ( selected ) >= shardsNeeded {
break
}
// Skip if already selected
alreadySelected := false
for _ , sel := range selected {
if sel . NodeID == disk . NodeID && sel . DiskID == disk . DiskID {
alreadySelected = true
break
}
}
if ! alreadySelected && isDiskSuitableForEC ( disk ) {
selected = append ( selected , disk )
}
}
}
return selected
}
// selectBestFromRack selects the best disk from a rack for EC placement
func selectBestFromRack ( disks [ ] * topology . DiskInfo , sourceRack , sourceDC string ) * topology . DiskInfo {
if len ( disks ) == 0 {
return nil
2025-07-30 12:38:03 -07:00
}
2025-08-01 11:18:32 -07:00
var bestDisk * topology . DiskInfo
bestScore := - 1.0
for _ , disk := range disks {
if ! isDiskSuitableForEC ( disk ) {
continue
}
score := calculateECScore ( disk , sourceRack , sourceDC )
if score > bestScore {
bestScore = score
bestDisk = disk
2025-07-30 12:38:03 -07:00
}
}
2025-08-01 11:18:32 -07:00
return bestDisk
}
// calculateECScore calculates placement score for EC operations
func calculateECScore ( disk * topology . DiskInfo , sourceRack , sourceDC string ) float64 {
if disk . DiskInfo == nil {
return 0.0
}
score := 0.0
2025-08-09 21:47:29 -07:00
// Prefer disks with available capacity (primary factor)
2025-08-01 11:18:32 -07:00
if disk . DiskInfo . MaxVolumeCount > 0 {
utilization := float64 ( disk . DiskInfo . VolumeCount ) / float64 ( disk . DiskInfo . MaxVolumeCount )
2025-08-09 21:47:29 -07:00
score += ( 1.0 - utilization ) * 60.0 // Up to 60 points for available capacity
2025-08-01 11:18:32 -07:00
}
2025-08-09 21:47:29 -07:00
// Consider current load (secondary factor)
2025-08-01 11:18:32 -07:00
score += ( 10.0 - float64 ( disk . LoadCount ) ) // Up to 10 points for low load
2025-08-09 21:47:29 -07:00
// Note: We don't penalize placing shards on the same rack/DC as source
// since the original volume will be deleted after EC conversion.
// This allows for better network efficiency and storage utilization.
2025-08-01 11:18:32 -07:00
return score
}
// isDiskSuitableForEC checks if a disk is suitable for EC placement
func isDiskSuitableForEC ( disk * topology . DiskInfo ) bool {
if disk . DiskInfo == nil {
return false
}
2025-08-03 01:35:38 -07:00
// Check if disk is not overloaded with tasks
if disk . LoadCount > topology . MaxTaskLoadForECPlacement {
2025-07-30 12:38:03 -07:00
return false
}
2025-08-01 11:18:32 -07:00
return true
}
2025-08-03 01:35:38 -07:00
// findVolumeReplicaLocations finds all replica locations (server + disk) for the specified volume
// Uses O(1) indexed lookup for optimal performance on large clusters.
func findVolumeReplicaLocations ( activeTopology * topology . ActiveTopology , volumeID uint32 , collection string ) [ ] topology . VolumeReplica {
if activeTopology == nil {
return nil
}
return activeTopology . GetVolumeLocations ( volumeID , collection )
}
// findExistingECShards finds existing EC shards for a volume (from previous failed EC attempts)
// Uses O(1) indexed lookup for optimal performance on large clusters.
func findExistingECShards ( activeTopology * topology . ActiveTopology , volumeID uint32 , collection string ) [ ] topology . VolumeReplica {
if activeTopology == nil {
return nil
}
return activeTopology . GetECShardLocations ( volumeID , collection )
}
2025-08-01 11:18:32 -07:00
// findVolumeReplicas finds all servers that have replicas of the specified volume
func findVolumeReplicas ( activeTopology * topology . ActiveTopology , volumeID uint32 , collection string ) [ ] string {
if activeTopology == nil {
return [ ] string { }
}
topologyInfo := activeTopology . GetTopologyInfo ( )
if topologyInfo == nil {
return [ ] string { }
}
var replicaServers [ ] string
// Iterate through all nodes to find volume replicas
for _ , dc := range topologyInfo . DataCenterInfos {
for _ , rack := range dc . RackInfos {
for _ , nodeInfo := range rack . DataNodeInfos {
for _ , diskInfo := range nodeInfo . DiskInfos {
for _ , volumeInfo := range diskInfo . VolumeInfos {
if volumeInfo . Id == volumeID && volumeInfo . Collection == collection {
replicaServers = append ( replicaServers , nodeInfo . Id )
break // Found volume on this node, move to next node
}
}
}
2025-07-30 12:38:03 -07:00
}
}
}
2025-08-01 11:18:32 -07:00
return replicaServers
2025-07-30 12:38:03 -07:00
}