mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2025-11-24 16:53:14 +08:00
admin: Refactor task destination planning (#7063)
* refactor planning into task detection * refactoring worker tasks * refactor * compiles, but only balance task is registered * compiles, but has nil exception * avoid nil logger * add back ec task * setting ec log directory * implement balance and vacuum tasks * EC tasks will no longer fail with "file not found" errors * Use ReceiveFile API to send locally generated shards * distributing shard files and ecx,ecj,vif files * generate .ecx files correctly * do not mount all possible EC shards (0-13) on every destination * use constants * delete all replicas * rename files * pass in volume size to tasks
This commit is contained in:
@@ -1,20 +1,13 @@
|
||||
package maintenance
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/seaweedfs/seaweedfs/weed/admin/topology"
|
||||
"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/worker_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/worker/tasks"
|
||||
"github.com/seaweedfs/seaweedfs/weed/worker/types"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
)
|
||||
|
||||
// MaintenanceIntegration bridges the task system with existing maintenance
|
||||
@@ -225,8 +218,9 @@ func (s *MaintenanceIntegration) ScanWithTaskDetectors(volumeMetrics []*types.Vo
|
||||
|
||||
// Create cluster info
|
||||
clusterInfo := &types.ClusterInfo{
|
||||
TotalVolumes: len(filteredMetrics),
|
||||
LastUpdated: time.Now(),
|
||||
TotalVolumes: len(filteredMetrics),
|
||||
LastUpdated: time.Now(),
|
||||
ActiveTopology: s.activeTopology, // Provide ActiveTopology for destination planning
|
||||
}
|
||||
|
||||
// Run detection for each registered task type
|
||||
@@ -250,8 +244,12 @@ func (s *MaintenanceIntegration) ScanWithTaskDetectors(volumeMetrics []*types.Vo
|
||||
// Double-check for conflicts with pending operations
|
||||
opType := s.mapMaintenanceTaskTypeToPendingOperationType(existingResult.TaskType)
|
||||
if !s.pendingOperations.WouldConflictWithPending(existingResult.VolumeID, opType) {
|
||||
// Plan destination for operations that need it
|
||||
s.planDestinationForTask(existingResult, opType)
|
||||
// All task types should now have TypedParams populated during detection phase
|
||||
if existingResult.TypedParams == nil {
|
||||
glog.Warningf("Task %s for volume %d has no typed parameters - skipping (task parameter creation may have failed)",
|
||||
existingResult.TaskType, existingResult.VolumeID)
|
||||
continue
|
||||
}
|
||||
allResults = append(allResults, existingResult)
|
||||
} else {
|
||||
glog.V(2).Infof("Skipping task %s for volume %d due to conflict with pending operation",
|
||||
@@ -342,7 +340,7 @@ func (s *MaintenanceIntegration) CanScheduleWithTaskSchedulers(task *Maintenance
|
||||
}
|
||||
|
||||
// convertTaskToTaskSystem converts existing task to task system format using dynamic mapping
|
||||
func (s *MaintenanceIntegration) convertTaskToTaskSystem(task *MaintenanceTask) *types.Task {
|
||||
func (s *MaintenanceIntegration) convertTaskToTaskSystem(task *MaintenanceTask) *types.TaskInput {
|
||||
// Convert task type using mapping
|
||||
taskType, exists := s.revTaskTypeMap[task.Type]
|
||||
if !exists {
|
||||
@@ -358,7 +356,7 @@ func (s *MaintenanceIntegration) convertTaskToTaskSystem(task *MaintenanceTask)
|
||||
priority = types.TaskPriorityNormal
|
||||
}
|
||||
|
||||
return &types.Task{
|
||||
return &types.TaskInput{
|
||||
ID: task.ID,
|
||||
Type: taskType,
|
||||
Priority: priority,
|
||||
@@ -371,8 +369,8 @@ func (s *MaintenanceIntegration) convertTaskToTaskSystem(task *MaintenanceTask)
|
||||
}
|
||||
|
||||
// convertTasksToTaskSystem converts multiple tasks
|
||||
func (s *MaintenanceIntegration) convertTasksToTaskSystem(tasks []*MaintenanceTask) []*types.Task {
|
||||
var result []*types.Task
|
||||
func (s *MaintenanceIntegration) convertTasksToTaskSystem(tasks []*MaintenanceTask) []*types.TaskInput {
|
||||
var result []*types.TaskInput
|
||||
for _, task := range tasks {
|
||||
converted := s.convertTaskToTaskSystem(task)
|
||||
if converted != nil {
|
||||
@@ -383,8 +381,8 @@ func (s *MaintenanceIntegration) convertTasksToTaskSystem(tasks []*MaintenanceTa
|
||||
}
|
||||
|
||||
// convertWorkersToTaskSystem converts workers to task system format using dynamic mapping
|
||||
func (s *MaintenanceIntegration) convertWorkersToTaskSystem(workers []*MaintenanceWorker) []*types.Worker {
|
||||
var result []*types.Worker
|
||||
func (s *MaintenanceIntegration) convertWorkersToTaskSystem(workers []*MaintenanceWorker) []*types.WorkerData {
|
||||
var result []*types.WorkerData
|
||||
for _, worker := range workers {
|
||||
capabilities := make([]types.TaskType, 0, len(worker.Capabilities))
|
||||
for _, cap := range worker.Capabilities {
|
||||
@@ -397,7 +395,7 @@ func (s *MaintenanceIntegration) convertWorkersToTaskSystem(workers []*Maintenan
|
||||
}
|
||||
}
|
||||
|
||||
result = append(result, &types.Worker{
|
||||
result = append(result, &types.WorkerData{
|
||||
ID: worker.ID,
|
||||
Address: worker.Address,
|
||||
Capabilities: capabilities,
|
||||
@@ -489,436 +487,3 @@ func (s *MaintenanceIntegration) GetPendingOperations() *PendingOperations {
|
||||
func (s *MaintenanceIntegration) GetActiveTopology() *topology.ActiveTopology {
|
||||
return s.activeTopology
|
||||
}
|
||||
|
||||
// planDestinationForTask plans the destination for a task that requires it and creates typed protobuf parameters
|
||||
func (s *MaintenanceIntegration) planDestinationForTask(task *TaskDetectionResult, opType PendingOperationType) {
|
||||
// Only plan destinations for operations that move volumes/shards
|
||||
if opType == OpTypeVacuum {
|
||||
// For vacuum tasks, create VacuumTaskParams
|
||||
s.createVacuumTaskParams(task)
|
||||
return
|
||||
}
|
||||
|
||||
glog.V(1).Infof("Planning destination for %s task on volume %d (server: %s)", task.TaskType, task.VolumeID, task.Server)
|
||||
|
||||
// Use ActiveTopology for destination planning
|
||||
destinationPlan, err := s.planDestinationWithActiveTopology(task, opType)
|
||||
|
||||
if err != nil {
|
||||
glog.Warningf("Failed to plan primary destination for %s task volume %d: %v",
|
||||
task.TaskType, task.VolumeID, err)
|
||||
// Don't return here - still try to create task params which might work with multiple destinations
|
||||
}
|
||||
|
||||
// Create typed protobuf parameters based on operation type
|
||||
switch opType {
|
||||
case OpTypeErasureCoding:
|
||||
if destinationPlan == nil {
|
||||
glog.Warningf("Cannot create EC task for volume %d: destination planning failed", task.VolumeID)
|
||||
return
|
||||
}
|
||||
s.createErasureCodingTaskParams(task, destinationPlan)
|
||||
case OpTypeVolumeMove, OpTypeVolumeBalance:
|
||||
if destinationPlan == nil {
|
||||
glog.Warningf("Cannot create balance task for volume %d: destination planning failed", task.VolumeID)
|
||||
return
|
||||
}
|
||||
s.createBalanceTaskParams(task, destinationPlan.(*topology.DestinationPlan))
|
||||
case OpTypeReplication:
|
||||
if destinationPlan == nil {
|
||||
glog.Warningf("Cannot create replication task for volume %d: destination planning failed", task.VolumeID)
|
||||
return
|
||||
}
|
||||
s.createReplicationTaskParams(task, destinationPlan.(*topology.DestinationPlan))
|
||||
default:
|
||||
glog.V(2).Infof("Unknown operation type for task %s: %v", task.TaskType, opType)
|
||||
}
|
||||
|
||||
if destinationPlan != nil {
|
||||
switch plan := destinationPlan.(type) {
|
||||
case *topology.DestinationPlan:
|
||||
glog.V(1).Infof("Completed destination planning for %s task on volume %d: %s -> %s",
|
||||
task.TaskType, task.VolumeID, task.Server, plan.TargetNode)
|
||||
case *topology.MultiDestinationPlan:
|
||||
glog.V(1).Infof("Completed EC destination planning for volume %d: %s -> %d destinations (racks: %d, DCs: %d)",
|
||||
task.VolumeID, task.Server, len(plan.Plans), plan.SuccessfulRack, plan.SuccessfulDCs)
|
||||
}
|
||||
} else {
|
||||
glog.V(1).Infof("Completed destination planning for %s task on volume %d: no destination planned",
|
||||
task.TaskType, task.VolumeID)
|
||||
}
|
||||
}
|
||||
|
||||
// createVacuumTaskParams creates typed parameters for vacuum tasks
|
||||
func (s *MaintenanceIntegration) createVacuumTaskParams(task *TaskDetectionResult) {
|
||||
// Get configuration from policy instead of using hard-coded values
|
||||
vacuumConfig := GetVacuumTaskConfig(s.maintenancePolicy, MaintenanceTaskType("vacuum"))
|
||||
|
||||
// Use configured values or defaults if config is not available
|
||||
garbageThreshold := 0.3 // Default 30%
|
||||
verifyChecksum := true // Default to verify
|
||||
batchSize := int32(1000) // Default batch size
|
||||
workingDir := "/tmp/seaweedfs_vacuum_work" // Default working directory
|
||||
|
||||
if vacuumConfig != nil {
|
||||
garbageThreshold = vacuumConfig.GarbageThreshold
|
||||
// Note: VacuumTaskConfig has GarbageThreshold, MinVolumeAgeHours, MinIntervalSeconds
|
||||
// Other fields like VerifyChecksum, BatchSize, WorkingDir would need to be added
|
||||
// to the protobuf definition if they should be configurable
|
||||
}
|
||||
|
||||
// Create typed protobuf parameters
|
||||
task.TypedParams = &worker_pb.TaskParams{
|
||||
VolumeId: task.VolumeID,
|
||||
Server: task.Server,
|
||||
Collection: task.Collection,
|
||||
TaskParams: &worker_pb.TaskParams_VacuumParams{
|
||||
VacuumParams: &worker_pb.VacuumTaskParams{
|
||||
GarbageThreshold: garbageThreshold,
|
||||
ForceVacuum: false,
|
||||
BatchSize: batchSize,
|
||||
WorkingDir: workingDir,
|
||||
VerifyChecksum: verifyChecksum,
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// planDestinationWithActiveTopology uses ActiveTopology to plan destinations
|
||||
func (s *MaintenanceIntegration) planDestinationWithActiveTopology(task *TaskDetectionResult, opType PendingOperationType) (interface{}, error) {
|
||||
// Get source node information from topology
|
||||
var sourceRack, sourceDC string
|
||||
|
||||
// Extract rack and DC from topology info
|
||||
topologyInfo := s.activeTopology.GetTopologyInfo()
|
||||
if topologyInfo != nil {
|
||||
for _, dc := range topologyInfo.DataCenterInfos {
|
||||
for _, rack := range dc.RackInfos {
|
||||
for _, dataNodeInfo := range rack.DataNodeInfos {
|
||||
if dataNodeInfo.Id == task.Server {
|
||||
sourceDC = dc.Id
|
||||
sourceRack = rack.Id
|
||||
break
|
||||
}
|
||||
}
|
||||
if sourceRack != "" {
|
||||
break
|
||||
}
|
||||
}
|
||||
if sourceDC != "" {
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
switch opType {
|
||||
case OpTypeVolumeBalance, OpTypeVolumeMove:
|
||||
// Plan single destination for balance operation
|
||||
return s.activeTopology.PlanBalanceDestination(task.VolumeID, task.Server, sourceRack, sourceDC, 0)
|
||||
|
||||
case OpTypeErasureCoding:
|
||||
// Plan multiple destinations for EC operation using adaptive shard counts
|
||||
// Start with the default configuration, but fall back to smaller configurations if insufficient disks
|
||||
totalShards := s.getOptimalECShardCount()
|
||||
multiPlan, err := s.activeTopology.PlanECDestinations(task.VolumeID, task.Server, sourceRack, sourceDC, totalShards)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if multiPlan != nil && len(multiPlan.Plans) > 0 {
|
||||
// Return the multi-destination plan for EC
|
||||
return multiPlan, nil
|
||||
}
|
||||
return nil, fmt.Errorf("no EC destinations found")
|
||||
|
||||
default:
|
||||
return nil, fmt.Errorf("unsupported operation type for destination planning: %v", opType)
|
||||
}
|
||||
}
|
||||
|
||||
// createErasureCodingTaskParams creates typed parameters for EC tasks
|
||||
func (s *MaintenanceIntegration) createErasureCodingTaskParams(task *TaskDetectionResult, destinationPlan interface{}) {
|
||||
// Determine EC shard counts based on the number of planned destinations
|
||||
multiPlan, ok := destinationPlan.(*topology.MultiDestinationPlan)
|
||||
if !ok {
|
||||
glog.Warningf("EC task for volume %d received unexpected destination plan type", task.VolumeID)
|
||||
task.TypedParams = nil
|
||||
return
|
||||
}
|
||||
|
||||
// Use adaptive shard configuration based on actual planned destinations
|
||||
totalShards := len(multiPlan.Plans)
|
||||
dataShards, parityShards := s.getECShardCounts(totalShards)
|
||||
|
||||
// Extract disk-aware destinations from the multi-destination plan
|
||||
var destinations []*worker_pb.ECDestination
|
||||
var allConflicts []string
|
||||
|
||||
for _, plan := range multiPlan.Plans {
|
||||
allConflicts = append(allConflicts, plan.Conflicts...)
|
||||
|
||||
// Create disk-aware destination
|
||||
destinations = append(destinations, &worker_pb.ECDestination{
|
||||
Node: plan.TargetNode,
|
||||
DiskId: plan.TargetDisk,
|
||||
Rack: plan.TargetRack,
|
||||
DataCenter: plan.TargetDC,
|
||||
PlacementScore: plan.PlacementScore,
|
||||
})
|
||||
}
|
||||
|
||||
glog.V(1).Infof("EC destination planning for volume %d: got %d destinations (%d+%d shards) across %d racks and %d DCs",
|
||||
task.VolumeID, len(destinations), dataShards, parityShards, multiPlan.SuccessfulRack, multiPlan.SuccessfulDCs)
|
||||
|
||||
if len(destinations) == 0 {
|
||||
glog.Warningf("No destinations available for EC task volume %d - rejecting task", task.VolumeID)
|
||||
task.TypedParams = nil
|
||||
return
|
||||
}
|
||||
|
||||
// Collect existing EC shard locations for cleanup
|
||||
existingShardLocations := s.collectExistingEcShardLocations(task.VolumeID)
|
||||
|
||||
// Create EC task parameters
|
||||
ecParams := &worker_pb.ErasureCodingTaskParams{
|
||||
Destinations: destinations, // Disk-aware destinations
|
||||
DataShards: dataShards,
|
||||
ParityShards: parityShards,
|
||||
WorkingDir: "/tmp/seaweedfs_ec_work",
|
||||
MasterClient: "localhost:9333",
|
||||
CleanupSource: true,
|
||||
ExistingShardLocations: existingShardLocations, // Pass existing shards for cleanup
|
||||
}
|
||||
|
||||
// Add placement conflicts if any
|
||||
if len(allConflicts) > 0 {
|
||||
// Remove duplicates
|
||||
conflictMap := make(map[string]bool)
|
||||
var uniqueConflicts []string
|
||||
for _, conflict := range allConflicts {
|
||||
if !conflictMap[conflict] {
|
||||
conflictMap[conflict] = true
|
||||
uniqueConflicts = append(uniqueConflicts, conflict)
|
||||
}
|
||||
}
|
||||
ecParams.PlacementConflicts = uniqueConflicts
|
||||
}
|
||||
|
||||
// Wrap in TaskParams
|
||||
task.TypedParams = &worker_pb.TaskParams{
|
||||
VolumeId: task.VolumeID,
|
||||
Server: task.Server,
|
||||
Collection: task.Collection,
|
||||
TaskParams: &worker_pb.TaskParams_ErasureCodingParams{
|
||||
ErasureCodingParams: ecParams,
|
||||
},
|
||||
}
|
||||
|
||||
glog.V(1).Infof("Created EC task params with %d destinations for volume %d",
|
||||
len(destinations), task.VolumeID)
|
||||
}
|
||||
|
||||
// createBalanceTaskParams creates typed parameters for balance/move tasks
|
||||
func (s *MaintenanceIntegration) createBalanceTaskParams(task *TaskDetectionResult, destinationPlan *topology.DestinationPlan) {
|
||||
// balanceConfig could be used for future config options like ImbalanceThreshold, MinServerCount
|
||||
|
||||
// Create balance task parameters
|
||||
balanceParams := &worker_pb.BalanceTaskParams{
|
||||
DestNode: destinationPlan.TargetNode,
|
||||
EstimatedSize: destinationPlan.ExpectedSize,
|
||||
DestRack: destinationPlan.TargetRack,
|
||||
DestDc: destinationPlan.TargetDC,
|
||||
PlacementScore: destinationPlan.PlacementScore,
|
||||
ForceMove: false, // Default to false
|
||||
TimeoutSeconds: 300, // Default 5 minutes
|
||||
}
|
||||
|
||||
// Add placement conflicts if any
|
||||
if len(destinationPlan.Conflicts) > 0 {
|
||||
balanceParams.PlacementConflicts = destinationPlan.Conflicts
|
||||
}
|
||||
|
||||
// Note: balanceConfig would have ImbalanceThreshold, MinServerCount if needed for future enhancements
|
||||
|
||||
// Wrap in TaskParams
|
||||
task.TypedParams = &worker_pb.TaskParams{
|
||||
VolumeId: task.VolumeID,
|
||||
Server: task.Server,
|
||||
Collection: task.Collection,
|
||||
TaskParams: &worker_pb.TaskParams_BalanceParams{
|
||||
BalanceParams: balanceParams,
|
||||
},
|
||||
}
|
||||
|
||||
glog.V(1).Infof("Created balance task params for volume %d: %s -> %s (score: %.2f)",
|
||||
task.VolumeID, task.Server, destinationPlan.TargetNode, destinationPlan.PlacementScore)
|
||||
}
|
||||
|
||||
// createReplicationTaskParams creates typed parameters for replication tasks
|
||||
func (s *MaintenanceIntegration) createReplicationTaskParams(task *TaskDetectionResult, destinationPlan *topology.DestinationPlan) {
|
||||
// replicationConfig could be used for future config options like TargetReplicaCount
|
||||
|
||||
// Create replication task parameters
|
||||
replicationParams := &worker_pb.ReplicationTaskParams{
|
||||
DestNode: destinationPlan.TargetNode,
|
||||
DestRack: destinationPlan.TargetRack,
|
||||
DestDc: destinationPlan.TargetDC,
|
||||
PlacementScore: destinationPlan.PlacementScore,
|
||||
}
|
||||
|
||||
// Add placement conflicts if any
|
||||
if len(destinationPlan.Conflicts) > 0 {
|
||||
replicationParams.PlacementConflicts = destinationPlan.Conflicts
|
||||
}
|
||||
|
||||
// Note: replicationConfig would have TargetReplicaCount if needed for future enhancements
|
||||
|
||||
// Wrap in TaskParams
|
||||
task.TypedParams = &worker_pb.TaskParams{
|
||||
VolumeId: task.VolumeID,
|
||||
Server: task.Server,
|
||||
Collection: task.Collection,
|
||||
TaskParams: &worker_pb.TaskParams_ReplicationParams{
|
||||
ReplicationParams: replicationParams,
|
||||
},
|
||||
}
|
||||
|
||||
glog.V(1).Infof("Created replication task params for volume %d: %s -> %s",
|
||||
task.VolumeID, task.Server, destinationPlan.TargetNode)
|
||||
}
|
||||
|
||||
// getOptimalECShardCount returns the optimal number of EC shards based on available disks
|
||||
// Uses a simplified approach to avoid blocking during UI access
|
||||
func (s *MaintenanceIntegration) getOptimalECShardCount() int {
|
||||
// Try to get available disks quickly, but don't block if topology is busy
|
||||
availableDisks := s.getAvailableDisksQuickly()
|
||||
|
||||
// EC configurations in order of preference: (data+parity=total)
|
||||
// Use smaller configurations for smaller clusters
|
||||
if availableDisks >= 14 {
|
||||
glog.V(1).Infof("Using default EC configuration: 10+4=14 shards for %d available disks", availableDisks)
|
||||
return 14 // Default: 10+4
|
||||
} else if availableDisks >= 6 {
|
||||
glog.V(1).Infof("Using small cluster EC configuration: 4+2=6 shards for %d available disks", availableDisks)
|
||||
return 6 // Small cluster: 4+2
|
||||
} else if availableDisks >= 4 {
|
||||
glog.V(1).Infof("Using minimal EC configuration: 3+1=4 shards for %d available disks", availableDisks)
|
||||
return 4 // Minimal: 3+1
|
||||
} else {
|
||||
glog.V(1).Infof("Using very small cluster EC configuration: 2+1=3 shards for %d available disks", availableDisks)
|
||||
return 3 // Very small: 2+1
|
||||
}
|
||||
}
|
||||
|
||||
// getAvailableDisksQuickly returns available disk count with a fast path to avoid UI blocking
|
||||
func (s *MaintenanceIntegration) getAvailableDisksQuickly() int {
|
||||
// Use ActiveTopology's optimized disk counting if available
|
||||
// Use empty task type and node filter for general availability check
|
||||
allDisks := s.activeTopology.GetAvailableDisks(topology.TaskTypeErasureCoding, "")
|
||||
if len(allDisks) > 0 {
|
||||
return len(allDisks)
|
||||
}
|
||||
|
||||
// Fallback: try to count from topology but don't hold locks for too long
|
||||
topologyInfo := s.activeTopology.GetTopologyInfo()
|
||||
return s.countAvailableDisks(topologyInfo)
|
||||
}
|
||||
|
||||
// countAvailableDisks counts the total number of available disks in the topology
|
||||
func (s *MaintenanceIntegration) countAvailableDisks(topologyInfo *master_pb.TopologyInfo) int {
|
||||
if topologyInfo == nil {
|
||||
return 0
|
||||
}
|
||||
|
||||
diskCount := 0
|
||||
for _, dc := range topologyInfo.DataCenterInfos {
|
||||
for _, rack := range dc.RackInfos {
|
||||
for _, node := range rack.DataNodeInfos {
|
||||
diskCount += len(node.DiskInfos)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return diskCount
|
||||
}
|
||||
|
||||
// getECShardCounts determines data and parity shard counts for a given total
|
||||
func (s *MaintenanceIntegration) getECShardCounts(totalShards int) (int32, int32) {
|
||||
// Map total shards to (data, parity) configurations
|
||||
switch totalShards {
|
||||
case 14:
|
||||
return 10, 4 // Default: 10+4
|
||||
case 9:
|
||||
return 6, 3 // Medium: 6+3
|
||||
case 6:
|
||||
return 4, 2 // Small: 4+2
|
||||
case 4:
|
||||
return 3, 1 // Minimal: 3+1
|
||||
case 3:
|
||||
return 2, 1 // Very small: 2+1
|
||||
default:
|
||||
// For any other total, try to maintain roughly 3:1 or 4:1 ratio
|
||||
if totalShards >= 4 {
|
||||
parityShards := totalShards / 4
|
||||
if parityShards < 1 {
|
||||
parityShards = 1
|
||||
}
|
||||
dataShards := totalShards - parityShards
|
||||
return int32(dataShards), int32(parityShards)
|
||||
}
|
||||
// Fallback for very small clusters
|
||||
return int32(totalShards - 1), 1
|
||||
}
|
||||
}
|
||||
|
||||
// collectExistingEcShardLocations queries the master for existing EC shard locations during planning
|
||||
func (s *MaintenanceIntegration) collectExistingEcShardLocations(volumeId uint32) []*worker_pb.ExistingECShardLocation {
|
||||
var existingShardLocations []*worker_pb.ExistingECShardLocation
|
||||
|
||||
// Use insecure connection for simplicity - in production this might be configurable
|
||||
grpcDialOption := grpc.WithTransportCredentials(insecure.NewCredentials())
|
||||
|
||||
err := operation.WithMasterServerClient(false, pb.ServerAddress("localhost:9333"), grpcDialOption,
|
||||
func(masterClient master_pb.SeaweedClient) error {
|
||||
req := &master_pb.LookupEcVolumeRequest{
|
||||
VolumeId: volumeId,
|
||||
}
|
||||
resp, err := masterClient.LookupEcVolume(context.Background(), req)
|
||||
if err != nil {
|
||||
// If volume doesn't exist as EC volume, that's fine - just no existing shards
|
||||
glog.V(1).Infof("LookupEcVolume for volume %d returned: %v (this is normal if no existing EC shards)", volumeId, err)
|
||||
return nil
|
||||
}
|
||||
|
||||
// Group shard locations by server
|
||||
serverShardMap := make(map[string][]uint32)
|
||||
for _, shardIdLocation := range resp.ShardIdLocations {
|
||||
shardId := uint32(shardIdLocation.ShardId)
|
||||
for _, location := range shardIdLocation.Locations {
|
||||
serverAddr := pb.NewServerAddressFromLocation(location)
|
||||
serverShardMap[string(serverAddr)] = append(serverShardMap[string(serverAddr)], shardId)
|
||||
}
|
||||
}
|
||||
|
||||
// Convert to protobuf format
|
||||
for serverAddr, shardIds := range serverShardMap {
|
||||
existingShardLocations = append(existingShardLocations, &worker_pb.ExistingECShardLocation{
|
||||
Node: serverAddr,
|
||||
ShardIds: shardIds,
|
||||
})
|
||||
}
|
||||
|
||||
return nil
|
||||
})
|
||||
|
||||
if err != nil {
|
||||
glog.Errorf("Failed to lookup existing EC shards from master for volume %d: %v", volumeId, err)
|
||||
// Return empty list - cleanup will be skipped but task can continue
|
||||
return []*worker_pb.ExistingECShardLocation{}
|
||||
}
|
||||
|
||||
if len(existingShardLocations) > 0 {
|
||||
glog.V(1).Infof("Found existing EC shards for volume %d on %d servers during planning", volumeId, len(existingShardLocations))
|
||||
}
|
||||
|
||||
return existingShardLocations
|
||||
}
|
||||
|
||||
@@ -73,20 +73,10 @@ func (ms *MaintenanceScanner) ScanForMaintenanceTasks() ([]*TaskDetectionResult,
|
||||
// getVolumeHealthMetrics collects health information for all volumes
|
||||
func (ms *MaintenanceScanner) getVolumeHealthMetrics() ([]*VolumeHealthMetrics, error) {
|
||||
var metrics []*VolumeHealthMetrics
|
||||
var volumeSizeLimitMB uint64
|
||||
|
||||
glog.V(1).Infof("Collecting volume health metrics from master")
|
||||
err := ms.adminClient.WithMasterClient(func(client master_pb.SeaweedClient) error {
|
||||
// First, get volume size limit from master configuration
|
||||
configResp, err := client.GetMasterConfiguration(context.Background(), &master_pb.GetMasterConfigurationRequest{})
|
||||
if err != nil {
|
||||
glog.Warningf("Failed to get volume size limit from master: %v", err)
|
||||
volumeSizeLimitMB = 30000 // Default to 30GB if we can't get from master
|
||||
} else {
|
||||
volumeSizeLimitMB = uint64(configResp.VolumeSizeLimitMB)
|
||||
}
|
||||
|
||||
// Now get volume list
|
||||
resp, err := client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
|
||||
if err != nil {
|
||||
return err
|
||||
@@ -97,7 +87,7 @@ func (ms *MaintenanceScanner) getVolumeHealthMetrics() ([]*VolumeHealthMetrics,
|
||||
return nil
|
||||
}
|
||||
|
||||
volumeSizeLimitBytes := volumeSizeLimitMB * 1024 * 1024 // Convert MB to bytes
|
||||
volumeSizeLimitBytes := uint64(resp.VolumeSizeLimitMb) * 1024 * 1024 // Convert MB to bytes
|
||||
|
||||
// Track all nodes discovered in topology
|
||||
var allNodesInTopology []string
|
||||
@@ -166,7 +156,6 @@ func (ms *MaintenanceScanner) getVolumeHealthMetrics() ([]*VolumeHealthMetrics,
|
||||
glog.Infof(" - Total volume servers in topology: %d (%v)", len(allNodesInTopology), allNodesInTopology)
|
||||
glog.Infof(" - Volume servers with volumes: %d (%v)", len(nodesWithVolumes), nodesWithVolumes)
|
||||
glog.Infof(" - Volume servers without volumes: %d (%v)", len(nodesWithoutVolumes), nodesWithoutVolumes)
|
||||
glog.Infof("Note: Maintenance system will track empty servers separately from volume metrics.")
|
||||
|
||||
// Store topology info for volume shard tracker
|
||||
ms.lastTopologyInfo = resp.TopologyInfo
|
||||
@@ -187,11 +176,6 @@ func (ms *MaintenanceScanner) getVolumeHealthMetrics() ([]*VolumeHealthMetrics,
|
||||
return metrics, nil
|
||||
}
|
||||
|
||||
// getTopologyInfo returns the last collected topology information
|
||||
func (ms *MaintenanceScanner) getTopologyInfo() *master_pb.TopologyInfo {
|
||||
return ms.lastTopologyInfo
|
||||
}
|
||||
|
||||
// enrichVolumeMetrics adds additional information like replica counts
|
||||
func (ms *MaintenanceScanner) enrichVolumeMetrics(metrics []*VolumeHealthMetrics) {
|
||||
// Group volumes by ID to count replicas
|
||||
|
||||
@@ -1,6 +1,7 @@
|
||||
package maintenance
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"os"
|
||||
"sync"
|
||||
@@ -131,13 +132,13 @@ func NewMaintenanceWorkerService(workerID, address, adminServer string) *Mainten
|
||||
currentTasks: make(map[string]*MaintenanceTask),
|
||||
stopChan: make(chan struct{}),
|
||||
taskExecutors: make(map[MaintenanceTaskType]TaskExecutor),
|
||||
taskRegistry: tasks.GetGlobalRegistry(), // Use global registry with auto-registered tasks
|
||||
taskRegistry: tasks.GetGlobalTaskRegistry(), // Use global registry with auto-registered tasks
|
||||
}
|
||||
|
||||
// Initialize task executor registry
|
||||
worker.initializeTaskExecutors()
|
||||
|
||||
glog.V(1).Infof("Created maintenance worker with %d registered task types", len(worker.taskRegistry.GetSupportedTypes()))
|
||||
glog.V(1).Infof("Created maintenance worker with %d registered task types", len(worker.taskRegistry.GetAll()))
|
||||
|
||||
return worker
|
||||
}
|
||||
@@ -154,16 +155,8 @@ func (mws *MaintenanceWorkerService) executeGenericTask(task *MaintenanceTask) e
|
||||
// Convert MaintenanceTask to types.TaskType
|
||||
taskType := types.TaskType(string(task.Type))
|
||||
|
||||
// Create task parameters
|
||||
taskParams := types.TaskParams{
|
||||
VolumeID: task.VolumeID,
|
||||
Server: task.Server,
|
||||
Collection: task.Collection,
|
||||
TypedParams: task.TypedParams,
|
||||
}
|
||||
|
||||
// Create task instance using the registry
|
||||
taskInstance, err := mws.taskRegistry.CreateTask(taskType, taskParams)
|
||||
taskInstance, err := mws.taskRegistry.Get(taskType).Create(task.TypedParams)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to create task instance: %w", err)
|
||||
}
|
||||
@@ -172,7 +165,7 @@ func (mws *MaintenanceWorkerService) executeGenericTask(task *MaintenanceTask) e
|
||||
mws.updateTaskProgress(task.ID, 5)
|
||||
|
||||
// Execute the task
|
||||
err = taskInstance.Execute(taskParams)
|
||||
err = taskInstance.Execute(context.Background(), task.TypedParams)
|
||||
if err != nil {
|
||||
return fmt.Errorf("task execution failed: %w", err)
|
||||
}
|
||||
|
||||
@@ -332,307 +332,6 @@ type MultiDestinationPlan struct {
|
||||
SuccessfulDCs int `json:"successful_dcs"`
|
||||
}
|
||||
|
||||
// PlanBalanceDestination finds the best destination for a balance operation
|
||||
func (at *ActiveTopology) PlanBalanceDestination(volumeID uint32, sourceNode string, sourceRack string, sourceDC string, volumeSize uint64) (*DestinationPlan, error) {
|
||||
at.mutex.RLock()
|
||||
defer at.mutex.RUnlock()
|
||||
|
||||
// Get available disks, excluding the source node
|
||||
availableDisks := at.getAvailableDisksForPlanning(TaskTypeBalance, sourceNode)
|
||||
if len(availableDisks) == 0 {
|
||||
return nil, fmt.Errorf("no available disks for balance operation")
|
||||
}
|
||||
|
||||
// Score each disk for balance placement
|
||||
bestDisk := at.selectBestBalanceDestination(availableDisks, sourceRack, sourceDC, volumeSize)
|
||||
if bestDisk == nil {
|
||||
return nil, fmt.Errorf("no suitable destination found for balance operation")
|
||||
}
|
||||
|
||||
return &DestinationPlan{
|
||||
TargetNode: bestDisk.NodeID,
|
||||
TargetDisk: bestDisk.DiskID,
|
||||
TargetRack: bestDisk.Rack,
|
||||
TargetDC: bestDisk.DataCenter,
|
||||
ExpectedSize: volumeSize,
|
||||
PlacementScore: at.calculatePlacementScore(bestDisk, sourceRack, sourceDC),
|
||||
Conflicts: at.checkPlacementConflicts(bestDisk, TaskTypeBalance),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// PlanECDestinations finds multiple destinations for EC shard distribution
|
||||
func (at *ActiveTopology) PlanECDestinations(volumeID uint32, sourceNode string, sourceRack string, sourceDC string, shardsNeeded int) (*MultiDestinationPlan, error) {
|
||||
at.mutex.RLock()
|
||||
defer at.mutex.RUnlock()
|
||||
|
||||
// Get available disks for EC placement
|
||||
availableDisks := at.getAvailableDisksForPlanning(TaskTypeErasureCoding, "")
|
||||
if len(availableDisks) < shardsNeeded {
|
||||
return nil, fmt.Errorf("insufficient disks for EC placement: need %d, have %d", shardsNeeded, len(availableDisks))
|
||||
}
|
||||
|
||||
// Select best disks for EC placement with rack/DC diversity
|
||||
selectedDisks := at.selectBestECDestinations(availableDisks, sourceRack, sourceDC, shardsNeeded)
|
||||
if len(selectedDisks) < shardsNeeded {
|
||||
return nil, fmt.Errorf("could not find %d suitable destinations for EC placement", shardsNeeded)
|
||||
}
|
||||
|
||||
var plans []*DestinationPlan
|
||||
rackCount := make(map[string]int)
|
||||
dcCount := make(map[string]int)
|
||||
|
||||
for _, disk := range selectedDisks {
|
||||
plan := &DestinationPlan{
|
||||
TargetNode: disk.NodeID,
|
||||
TargetDisk: disk.DiskID,
|
||||
TargetRack: disk.Rack,
|
||||
TargetDC: disk.DataCenter,
|
||||
ExpectedSize: 0, // EC shards don't have predetermined size
|
||||
PlacementScore: at.calculatePlacementScore(disk, sourceRack, sourceDC),
|
||||
Conflicts: at.checkPlacementConflicts(disk, TaskTypeErasureCoding),
|
||||
}
|
||||
plans = append(plans, plan)
|
||||
|
||||
// Count rack and DC diversity
|
||||
rackKey := fmt.Sprintf("%s:%s", disk.DataCenter, disk.Rack)
|
||||
rackCount[rackKey]++
|
||||
dcCount[disk.DataCenter]++
|
||||
}
|
||||
|
||||
return &MultiDestinationPlan{
|
||||
Plans: plans,
|
||||
TotalShards: len(plans),
|
||||
SuccessfulRack: len(rackCount),
|
||||
SuccessfulDCs: len(dcCount),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// getAvailableDisksForPlanning returns disks available for destination planning
|
||||
func (at *ActiveTopology) getAvailableDisksForPlanning(taskType TaskType, excludeNodeID string) []*activeDisk {
|
||||
var available []*activeDisk
|
||||
|
||||
for _, disk := range at.disks {
|
||||
if excludeNodeID != "" && disk.NodeID == excludeNodeID {
|
||||
continue // Skip excluded node
|
||||
}
|
||||
|
||||
if at.isDiskAvailable(disk, taskType) {
|
||||
available = append(available, disk)
|
||||
}
|
||||
}
|
||||
|
||||
return available
|
||||
}
|
||||
|
||||
// selectBestBalanceDestination selects the best disk for balance operation
|
||||
func (at *ActiveTopology) selectBestBalanceDestination(disks []*activeDisk, sourceRack string, sourceDC string, volumeSize uint64) *activeDisk {
|
||||
if len(disks) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
var bestDisk *activeDisk
|
||||
bestScore := -1.0
|
||||
|
||||
for _, disk := range disks {
|
||||
score := at.calculateBalanceScore(disk, sourceRack, sourceDC, volumeSize)
|
||||
if score > bestScore {
|
||||
bestScore = score
|
||||
bestDisk = disk
|
||||
}
|
||||
}
|
||||
|
||||
return bestDisk
|
||||
}
|
||||
|
||||
// selectBestECDestinations selects multiple disks for EC shard placement with diversity
|
||||
func (at *ActiveTopology) selectBestECDestinations(disks []*activeDisk, sourceRack string, sourceDC string, shardsNeeded int) []*activeDisk {
|
||||
if len(disks) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
// Group disks by rack and DC for diversity
|
||||
rackGroups := make(map[string][]*activeDisk)
|
||||
for _, disk := range disks {
|
||||
rackKey := fmt.Sprintf("%s:%s", disk.DataCenter, disk.Rack)
|
||||
rackGroups[rackKey] = append(rackGroups[rackKey], disk)
|
||||
}
|
||||
|
||||
var selected []*activeDisk
|
||||
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 := at.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 && at.isDiskAvailable(disk, TaskTypeErasureCoding) {
|
||||
selected = append(selected, disk)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return selected
|
||||
}
|
||||
|
||||
// selectBestFromRack selects the best disk from a rack
|
||||
func (at *ActiveTopology) selectBestFromRack(disks []*activeDisk, sourceRack string, sourceDC string) *activeDisk {
|
||||
if len(disks) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
var bestDisk *activeDisk
|
||||
bestScore := -1.0
|
||||
|
||||
for _, disk := range disks {
|
||||
if !at.isDiskAvailable(disk, TaskTypeErasureCoding) {
|
||||
continue
|
||||
}
|
||||
|
||||
score := at.calculateECScore(disk, sourceRack, sourceDC)
|
||||
if score > bestScore {
|
||||
bestScore = score
|
||||
bestDisk = disk
|
||||
}
|
||||
}
|
||||
|
||||
return bestDisk
|
||||
}
|
||||
|
||||
// calculateBalanceScore calculates placement score for balance operations
|
||||
func (at *ActiveTopology) calculateBalanceScore(disk *activeDisk, sourceRack string, sourceDC string, volumeSize uint64) float64 {
|
||||
score := 0.0
|
||||
|
||||
// Prefer disks with lower load
|
||||
activeLoad := len(disk.pendingTasks) + len(disk.assignedTasks)
|
||||
score += (2.0 - float64(activeLoad)) * 40.0 // Max 80 points for load
|
||||
|
||||
// Prefer disks with more free space
|
||||
if disk.DiskInfo.DiskInfo.MaxVolumeCount > 0 {
|
||||
freeRatio := float64(disk.DiskInfo.DiskInfo.MaxVolumeCount-disk.DiskInfo.DiskInfo.VolumeCount) / float64(disk.DiskInfo.DiskInfo.MaxVolumeCount)
|
||||
score += freeRatio * 20.0 // Max 20 points for free space
|
||||
}
|
||||
|
||||
// Rack diversity bonus (prefer different rack)
|
||||
if disk.Rack != sourceRack {
|
||||
score += 10.0
|
||||
}
|
||||
|
||||
// DC diversity bonus (prefer different DC)
|
||||
if disk.DataCenter != sourceDC {
|
||||
score += 5.0
|
||||
}
|
||||
|
||||
return score
|
||||
}
|
||||
|
||||
// calculateECScore calculates placement score for EC operations
|
||||
func (at *ActiveTopology) calculateECScore(disk *activeDisk, sourceRack string, sourceDC string) float64 {
|
||||
score := 0.0
|
||||
|
||||
// Prefer disks with lower load
|
||||
activeLoad := len(disk.pendingTasks) + len(disk.assignedTasks)
|
||||
score += (2.0 - float64(activeLoad)) * 30.0 // Max 60 points for load
|
||||
|
||||
// Prefer disks with more free space
|
||||
if disk.DiskInfo.DiskInfo.MaxVolumeCount > 0 {
|
||||
freeRatio := float64(disk.DiskInfo.DiskInfo.MaxVolumeCount-disk.DiskInfo.DiskInfo.VolumeCount) / float64(disk.DiskInfo.DiskInfo.MaxVolumeCount)
|
||||
score += freeRatio * 20.0 // Max 20 points for free space
|
||||
}
|
||||
|
||||
// Strong rack diversity preference for EC
|
||||
if disk.Rack != sourceRack {
|
||||
score += 20.0
|
||||
}
|
||||
|
||||
// Strong DC diversity preference for EC
|
||||
if disk.DataCenter != sourceDC {
|
||||
score += 15.0
|
||||
}
|
||||
|
||||
return score
|
||||
}
|
||||
|
||||
// calculatePlacementScore calculates overall placement quality score
|
||||
func (at *ActiveTopology) calculatePlacementScore(disk *activeDisk, sourceRack string, sourceDC string) float64 {
|
||||
score := 0.0
|
||||
|
||||
// Load factor
|
||||
activeLoad := len(disk.pendingTasks) + len(disk.assignedTasks)
|
||||
loadScore := (2.0 - float64(activeLoad)) / 2.0 // Normalize to 0-1
|
||||
score += loadScore * 0.4
|
||||
|
||||
// Capacity factor
|
||||
if disk.DiskInfo.DiskInfo.MaxVolumeCount > 0 {
|
||||
freeRatio := float64(disk.DiskInfo.DiskInfo.MaxVolumeCount-disk.DiskInfo.DiskInfo.VolumeCount) / float64(disk.DiskInfo.DiskInfo.MaxVolumeCount)
|
||||
score += freeRatio * 0.3
|
||||
}
|
||||
|
||||
// Diversity factor
|
||||
diversityScore := 0.0
|
||||
if disk.Rack != sourceRack {
|
||||
diversityScore += 0.5
|
||||
}
|
||||
if disk.DataCenter != sourceDC {
|
||||
diversityScore += 0.5
|
||||
}
|
||||
score += diversityScore * 0.3
|
||||
|
||||
return score // Score between 0.0 and 1.0
|
||||
}
|
||||
|
||||
// checkPlacementConflicts checks for placement rule violations
|
||||
func (at *ActiveTopology) checkPlacementConflicts(disk *activeDisk, taskType TaskType) []string {
|
||||
var conflicts []string
|
||||
|
||||
// Check load limits
|
||||
activeLoad := len(disk.pendingTasks) + len(disk.assignedTasks)
|
||||
if activeLoad >= 2 {
|
||||
conflicts = append(conflicts, fmt.Sprintf("disk_load_high_%d", activeLoad))
|
||||
}
|
||||
|
||||
// Check capacity limits
|
||||
if disk.DiskInfo.DiskInfo.MaxVolumeCount > 0 {
|
||||
usageRatio := float64(disk.DiskInfo.DiskInfo.VolumeCount) / float64(disk.DiskInfo.DiskInfo.MaxVolumeCount)
|
||||
if usageRatio > 0.9 {
|
||||
conflicts = append(conflicts, "disk_capacity_high")
|
||||
}
|
||||
}
|
||||
|
||||
// Check for conflicting task types
|
||||
for _, task := range disk.assignedTasks {
|
||||
if at.areTaskTypesConflicting(task.TaskType, taskType) {
|
||||
conflicts = append(conflicts, fmt.Sprintf("task_conflict_%s", task.TaskType))
|
||||
}
|
||||
}
|
||||
|
||||
return conflicts
|
||||
}
|
||||
|
||||
// Private methods
|
||||
|
||||
// reassignTaskStates assigns tasks to the appropriate disks
|
||||
|
||||
@@ -4,7 +4,6 @@ import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
@@ -478,177 +477,31 @@ func createTopologyWithConflicts() *ActiveTopology {
|
||||
return topology
|
||||
}
|
||||
|
||||
// TestDestinationPlanning tests destination planning functionality
|
||||
// TestDestinationPlanning tests that the public interface works correctly
|
||||
// NOTE: Destination planning is now done in task detection phase, not in ActiveTopology
|
||||
func TestDestinationPlanning(t *testing.T) {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createSampleTopology())
|
||||
|
||||
// Test balance destination planning
|
||||
t.Run("Balance destination planning", func(t *testing.T) {
|
||||
plan, err := topology.PlanBalanceDestination(1001, "10.0.0.1:8080", "rack1", "dc1", 1024*1024) // 1MB
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, plan)
|
||||
// Test that GetAvailableDisks works for destination planning
|
||||
t.Run("GetAvailableDisks functionality", func(t *testing.T) {
|
||||
availableDisks := topology.GetAvailableDisks(TaskTypeBalance, "10.0.0.1:8080")
|
||||
assert.Greater(t, len(availableDisks), 0)
|
||||
|
||||
// Should not target the source node
|
||||
assert.NotEqual(t, "10.0.0.1:8080", plan.TargetNode)
|
||||
assert.Equal(t, "10.0.0.2:8080", plan.TargetNode)
|
||||
assert.NotEmpty(t, plan.TargetRack)
|
||||
assert.NotEmpty(t, plan.TargetDC)
|
||||
assert.Greater(t, plan.PlacementScore, 0.0)
|
||||
})
|
||||
|
||||
// Test EC destination planning
|
||||
t.Run("EC destination planning", func(t *testing.T) {
|
||||
multiPlan, err := topology.PlanECDestinations(1002, "10.0.0.1:8080", "rack1", "dc1", 3) // Ask for 3 shards - source node can be included
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, multiPlan)
|
||||
assert.Greater(t, len(multiPlan.Plans), 0)
|
||||
assert.LessOrEqual(t, len(multiPlan.Plans), 3) // Should get at most 3 shards
|
||||
assert.Equal(t, len(multiPlan.Plans), multiPlan.TotalShards)
|
||||
|
||||
// Check that all plans have valid target nodes
|
||||
for _, plan := range multiPlan.Plans {
|
||||
assert.NotEmpty(t, plan.TargetNode)
|
||||
assert.NotEmpty(t, plan.TargetRack)
|
||||
assert.NotEmpty(t, plan.TargetDC)
|
||||
assert.GreaterOrEqual(t, plan.PlacementScore, 0.0)
|
||||
// Should exclude the source node
|
||||
for _, disk := range availableDisks {
|
||||
assert.NotEqual(t, "10.0.0.1:8080", disk.NodeID)
|
||||
}
|
||||
|
||||
// Check diversity metrics
|
||||
assert.GreaterOrEqual(t, multiPlan.SuccessfulRack, 1)
|
||||
assert.GreaterOrEqual(t, multiPlan.SuccessfulDCs, 1)
|
||||
})
|
||||
|
||||
// Test destination planning with load
|
||||
t.Run("Destination planning considers load", func(t *testing.T) {
|
||||
// Add load to one disk
|
||||
topology.AddPendingTask("task1", TaskTypeBalance, 2001,
|
||||
"10.0.0.2:8080", 0, "", 0)
|
||||
// Test that topology state can be used for planning
|
||||
t.Run("Topology provides planning information", func(t *testing.T) {
|
||||
topologyInfo := topology.GetTopologyInfo()
|
||||
assert.NotNil(t, topologyInfo)
|
||||
assert.Greater(t, len(topologyInfo.DataCenterInfos), 0)
|
||||
|
||||
plan, err := topology.PlanBalanceDestination(1003, "10.0.0.1:8080", "rack1", "dc1", 1024*1024)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, plan)
|
||||
|
||||
// Should prefer less loaded disk (disk 1 over disk 0 on node2)
|
||||
assert.Equal(t, "10.0.0.2:8080", plan.TargetNode)
|
||||
assert.Equal(t, uint32(1), plan.TargetDisk) // Should prefer SSD (disk 1) which has no load
|
||||
})
|
||||
|
||||
// Test insufficient destinations
|
||||
t.Run("Handle insufficient destinations", func(t *testing.T) {
|
||||
// Try to plan for more EC shards than available disks
|
||||
multiPlan, err := topology.PlanECDestinations(1004, "10.0.0.1:8080", "rack1", "dc1", 100)
|
||||
|
||||
// Should get an error for insufficient disks
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, multiPlan)
|
||||
})
|
||||
}
|
||||
|
||||
// TestDestinationPlanningWithActiveTopology tests the integration between task detection and destination planning
|
||||
func TestDestinationPlanningWithActiveTopology(t *testing.T) {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createUnbalancedTopology())
|
||||
|
||||
// Test that tasks are created with destinations
|
||||
t.Run("Balance task with destination", func(t *testing.T) {
|
||||
// Simulate what the balance detector would create
|
||||
sourceNode := "10.0.0.1:8080" // Overloaded node
|
||||
volumeID := uint32(1001)
|
||||
|
||||
plan, err := topology.PlanBalanceDestination(volumeID, sourceNode, "rack1", "dc1", 1024*1024)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, plan)
|
||||
|
||||
// Verify the destination is different from source
|
||||
assert.NotEqual(t, sourceNode, plan.TargetNode)
|
||||
assert.Equal(t, "10.0.0.2:8080", plan.TargetNode) // Should be the lightly loaded node
|
||||
|
||||
// Verify placement quality
|
||||
assert.Greater(t, plan.PlacementScore, 0.0)
|
||||
assert.LessOrEqual(t, plan.PlacementScore, 1.0)
|
||||
})
|
||||
|
||||
// Test task state integration
|
||||
t.Run("Task state affects future planning", func(t *testing.T) {
|
||||
volumeID := uint32(1002)
|
||||
sourceNode := "10.0.0.1:8080"
|
||||
targetNode := "10.0.0.2:8080"
|
||||
|
||||
// Plan first destination
|
||||
plan1, err := topology.PlanBalanceDestination(volumeID, sourceNode, "rack1", "dc1", 1024*1024)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, plan1)
|
||||
|
||||
// Add a pending task to the target
|
||||
topology.AddPendingTask("task1", TaskTypeBalance, volumeID, sourceNode, 0, targetNode, 0)
|
||||
|
||||
// Plan another destination - should consider the pending task load
|
||||
plan2, err := topology.PlanBalanceDestination(1003, sourceNode, "rack1", "dc1", 1024*1024)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, plan2)
|
||||
|
||||
// The placement score should reflect the increased load
|
||||
// (This test might need adjustment based on the actual scoring algorithm)
|
||||
glog.V(1).Infof("Plan1 score: %.3f, Plan2 score: %.3f", plan1.PlacementScore, plan2.PlacementScore)
|
||||
})
|
||||
}
|
||||
|
||||
// TestECDestinationPlanningDetailed tests the EC destination planning with multiple shards
|
||||
func TestECDestinationPlanningDetailed(t *testing.T) {
|
||||
topology := NewActiveTopology(10)
|
||||
topology.UpdateTopology(createSampleTopology())
|
||||
|
||||
t.Run("EC multiple destinations", func(t *testing.T) {
|
||||
// Plan for 3 EC shards (now including source node, we have 4 disks total)
|
||||
multiPlan, err := topology.PlanECDestinations(1005, "10.0.0.1:8080", "rack1", "dc1", 3)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, multiPlan)
|
||||
|
||||
// Should get 3 destinations (can include source node's disks)
|
||||
assert.Equal(t, 3, len(multiPlan.Plans))
|
||||
assert.Equal(t, 3, multiPlan.TotalShards)
|
||||
|
||||
// Count node distribution - source node can now be included
|
||||
nodeCount := make(map[string]int)
|
||||
for _, plan := range multiPlan.Plans {
|
||||
nodeCount[plan.TargetNode]++
|
||||
}
|
||||
|
||||
// Should distribute across available nodes (both nodes can be used)
|
||||
assert.GreaterOrEqual(t, len(nodeCount), 1, "Should use at least 1 node")
|
||||
assert.LessOrEqual(t, len(nodeCount), 2, "Should use at most 2 nodes")
|
||||
glog.V(1).Infof("EC destinations node distribution: %v", nodeCount)
|
||||
|
||||
glog.V(1).Infof("EC destinations: %d plans across %d racks, %d DCs",
|
||||
multiPlan.TotalShards, multiPlan.SuccessfulRack, multiPlan.SuccessfulDCs)
|
||||
})
|
||||
|
||||
t.Run("EC destination planning with task conflicts", func(t *testing.T) {
|
||||
// Create a fresh topology for this test to avoid conflicts from previous test
|
||||
freshTopology := NewActiveTopology(10)
|
||||
freshTopology.UpdateTopology(createSampleTopology())
|
||||
|
||||
// Add tasks to create conflicts on some disks
|
||||
freshTopology.AddPendingTask("conflict1", TaskTypeVacuum, 2001, "10.0.0.2:8080", 0, "", 0)
|
||||
freshTopology.AddPendingTask("conflict2", TaskTypeBalance, 2002, "10.0.0.1:8080", 0, "", 0)
|
||||
freshTopology.AssignTask("conflict1")
|
||||
freshTopology.AssignTask("conflict2")
|
||||
|
||||
// Plan EC destinations - should still succeed using available disks
|
||||
multiPlan, err := freshTopology.PlanECDestinations(1006, "10.0.0.1:8080", "rack1", "dc1", 2)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, multiPlan)
|
||||
|
||||
// Should get destinations (using disks that don't have conflicts)
|
||||
assert.GreaterOrEqual(t, len(multiPlan.Plans), 1)
|
||||
assert.LessOrEqual(t, len(multiPlan.Plans), 2)
|
||||
|
||||
// Available disks should be: node1/disk1 and node2/disk1 (since disk0 on both nodes have conflicts)
|
||||
for _, plan := range multiPlan.Plans {
|
||||
assert.Equal(t, uint32(1), plan.TargetDisk, "Should prefer disk 1 which has no conflicts")
|
||||
}
|
||||
|
||||
glog.V(1).Infof("EC destination planning with conflicts: found %d destinations", len(multiPlan.Plans))
|
||||
// Test getting node disks
|
||||
disks := topology.GetNodeDisks("10.0.0.1:8080")
|
||||
assert.Greater(t, len(disks), 0)
|
||||
})
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user