From 0975968e71b05368d5f28f788cf863c2042c2696 Mon Sep 17 00:00:00 2001 From: Chris Lu Date: Fri, 1 Aug 2025 11:18:32 -0700 Subject: [PATCH] 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 --- .../docker-compose-ec-test.yml | 2 +- .../maintenance/maintenance_integration.go | 467 +---------- weed/admin/maintenance/maintenance_scanner.go | 18 +- weed/admin/maintenance/maintenance_worker.go | 17 +- weed/admin/topology/active_topology.go | 301 ------- weed/admin/topology/active_topology_test.go | 181 +--- weed/pb/worker.proto | 1 + weed/pb/worker_pb/worker.pb.go | 14 +- weed/worker/client.go | 22 +- weed/worker/registry.go | 28 +- weed/worker/tasks/balance/balance.go | 141 ---- weed/worker/tasks/balance/balance_task.go | 248 ++++++ weed/worker/tasks/balance/detection.go | 154 +++- .../{balance_typed.go => execution.go} | 0 weed/worker/tasks/balance/monitoring.go | 138 +++ .../{balance_register.go => register.go} | 17 +- weed/worker/tasks/balance/scheduling.go | 37 + weed/worker/tasks/base/generic_components.go | 6 +- weed/worker/tasks/base/registration.go | 22 +- weed/worker/tasks/base/task_definition.go | 4 +- weed/worker/tasks/erasure_coding/detection.go | 314 ++++++- weed/worker/tasks/erasure_coding/ec.go | 785 ------------------ weed/worker/tasks/erasure_coding/ec_task.go | 660 +++++++++++++++ .../worker/tasks/erasure_coding/monitoring.go | 229 +++++ .../{ec_register.go => register.go} | 17 +- .../worker/tasks/erasure_coding/scheduling.go | 40 + weed/worker/tasks/registry.go | 64 +- weed/worker/tasks/task.go | 53 -- weed/worker/tasks/vacuum/detection.go | 65 +- weed/worker/tasks/vacuum/monitoring.go | 151 ++++ .../{vacuum_register.go => register.go} | 17 +- weed/worker/tasks/vacuum/scheduling.go | 37 + weed/worker/tasks/vacuum/vacuum.go | 214 ----- weed/worker/tasks/vacuum/vacuum_task.go | 236 ++++++ weed/worker/types/base/task.go | 99 +++ weed/worker/types/data_types.go | 11 +- weed/worker/types/factory.go | 52 ++ weed/worker/types/task.go | 189 +++++ weed/worker/types/task_scheduler.go | 4 +- weed/worker/types/task_types.go | 4 +- weed/worker/types/worker.go | 76 ++ weed/worker/types/worker_types.go | 53 +- weed/worker/worker.go | 89 +- 43 files changed, 2901 insertions(+), 2376 deletions(-) delete mode 100644 weed/worker/tasks/balance/balance.go create mode 100644 weed/worker/tasks/balance/balance_task.go rename weed/worker/tasks/balance/{balance_typed.go => execution.go} (100%) create mode 100644 weed/worker/tasks/balance/monitoring.go rename weed/worker/tasks/balance/{balance_register.go => register.go} (82%) create mode 100644 weed/worker/tasks/balance/scheduling.go delete mode 100644 weed/worker/tasks/erasure_coding/ec.go create mode 100644 weed/worker/tasks/erasure_coding/ec_task.go create mode 100644 weed/worker/tasks/erasure_coding/monitoring.go rename weed/worker/tasks/erasure_coding/{ec_register.go => register.go} (83%) create mode 100644 weed/worker/tasks/erasure_coding/scheduling.go create mode 100644 weed/worker/tasks/vacuum/monitoring.go rename weed/worker/tasks/vacuum/{vacuum_register.go => register.go} (82%) create mode 100644 weed/worker/tasks/vacuum/scheduling.go delete mode 100644 weed/worker/tasks/vacuum/vacuum.go create mode 100644 weed/worker/tasks/vacuum/vacuum_task.go create mode 100644 weed/worker/types/base/task.go create mode 100644 weed/worker/types/factory.go create mode 100644 weed/worker/types/task.go create mode 100644 weed/worker/types/worker.go diff --git a/docker/admin_integration/docker-compose-ec-test.yml b/docker/admin_integration/docker-compose-ec-test.yml index 197c9bda5..73d0ee0ff 100644 --- a/docker/admin_integration/docker-compose-ec-test.yml +++ b/docker/admin_integration/docker-compose-ec-test.yml @@ -116,7 +116,7 @@ services: ports: - "23646:23646" # HTTP admin interface (default port) - "33646:33646" # gRPC worker communication (23646 + 10000) - command: "admin -port=23646 -masters=master:9333 -dataDir=/data" + command: "-v=2 admin -port=23646 -masters=master:9333 -dataDir=/data" depends_on: - master - filer diff --git a/weed/admin/maintenance/maintenance_integration.go b/weed/admin/maintenance/maintenance_integration.go index 1bdd7ffcc..553f32eb8 100644 --- a/weed/admin/maintenance/maintenance_integration.go +++ b/weed/admin/maintenance/maintenance_integration.go @@ -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 -} diff --git a/weed/admin/maintenance/maintenance_scanner.go b/weed/admin/maintenance/maintenance_scanner.go index ef41b78ed..3f8a528eb 100644 --- a/weed/admin/maintenance/maintenance_scanner.go +++ b/weed/admin/maintenance/maintenance_scanner.go @@ -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 diff --git a/weed/admin/maintenance/maintenance_worker.go b/weed/admin/maintenance/maintenance_worker.go index 96e17f9e9..e4a6b4cf6 100644 --- a/weed/admin/maintenance/maintenance_worker.go +++ b/weed/admin/maintenance/maintenance_worker.go @@ -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) } diff --git a/weed/admin/topology/active_topology.go b/weed/admin/topology/active_topology.go index 9ce63bfa7..bfa03a72f 100644 --- a/weed/admin/topology/active_topology.go +++ b/weed/admin/topology/active_topology.go @@ -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 diff --git a/weed/admin/topology/active_topology_test.go b/weed/admin/topology/active_topology_test.go index 9f2f09c29..4e8b0b3a8 100644 --- a/weed/admin/topology/active_topology_test.go +++ b/weed/admin/topology/active_topology_test.go @@ -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) - - // 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 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) - // 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) - - 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") - } + // 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) - 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) }) } diff --git a/weed/pb/worker.proto b/weed/pb/worker.proto index 63eeea4b0..0ab115bb2 100644 --- a/weed/pb/worker.proto +++ b/weed/pb/worker.proto @@ -100,6 +100,7 @@ message TaskParams { string data_center = 4; string rack = 5; repeated string replicas = 6; + uint64 volume_size = 11; // Original volume size in bytes for tracking size changes // Typed task parameters oneof task_params { diff --git a/weed/pb/worker_pb/worker.pb.go b/weed/pb/worker_pb/worker.pb.go index 210f6feac..f6b3e9fb1 100644 --- a/weed/pb/worker_pb/worker.pb.go +++ b/weed/pb/worker_pb/worker.pb.go @@ -810,6 +810,7 @@ type TaskParams struct { DataCenter string `protobuf:"bytes,4,opt,name=data_center,json=dataCenter,proto3" json:"data_center,omitempty"` Rack string `protobuf:"bytes,5,opt,name=rack,proto3" json:"rack,omitempty"` Replicas []string `protobuf:"bytes,6,rep,name=replicas,proto3" json:"replicas,omitempty"` + VolumeSize uint64 `protobuf:"varint,11,opt,name=volume_size,json=volumeSize,proto3" json:"volume_size,omitempty"` // Original volume size in bytes for tracking size changes // Typed task parameters // // Types that are valid to be assigned to TaskParams: @@ -895,6 +896,13 @@ func (x *TaskParams) GetReplicas() []string { return nil } +func (x *TaskParams) GetVolumeSize() uint64 { + if x != nil { + return x.VolumeSize + } + return 0 +} + func (x *TaskParams) GetTaskParams() isTaskParams_TaskParams { if x != nil { return x.TaskParams @@ -2861,7 +2869,7 @@ const file_worker_proto_rawDesc = "" + "\bmetadata\x18\x06 \x03(\v2'.worker_pb.TaskAssignment.MetadataEntryR\bmetadata\x1a;\n" + "\rMetadataEntry\x12\x10\n" + "\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n" + - "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\xf9\x03\n" + + "\x05value\x18\x02 \x01(\tR\x05value:\x028\x01\"\x9a\x04\n" + "\n" + "TaskParams\x12\x1b\n" + "\tvolume_id\x18\x01 \x01(\rR\bvolumeId\x12\x16\n" + @@ -2872,7 +2880,9 @@ const file_worker_proto_rawDesc = "" + "\vdata_center\x18\x04 \x01(\tR\n" + "dataCenter\x12\x12\n" + "\x04rack\x18\x05 \x01(\tR\x04rack\x12\x1a\n" + - "\breplicas\x18\x06 \x03(\tR\breplicas\x12B\n" + + "\breplicas\x18\x06 \x03(\tR\breplicas\x12\x1f\n" + + "\vvolume_size\x18\v \x01(\x04R\n" + + "volumeSize\x12B\n" + "\rvacuum_params\x18\a \x01(\v2\x1b.worker_pb.VacuumTaskParamsH\x00R\fvacuumParams\x12X\n" + "\x15erasure_coding_params\x18\b \x01(\v2\".worker_pb.ErasureCodingTaskParamsH\x00R\x13erasureCodingParams\x12E\n" + "\x0ebalance_params\x18\t \x01(\v2\x1c.worker_pb.BalanceTaskParamsH\x00R\rbalanceParams\x12Q\n" + diff --git a/weed/worker/client.go b/weed/worker/client.go index 53854c6e3..ef7e431c0 100644 --- a/weed/worker/client.go +++ b/weed/worker/client.go @@ -38,7 +38,7 @@ type GrpcAdminClient struct { reconnectMultiplier float64 // Worker registration info for re-registration after reconnection - lastWorkerInfo *types.Worker + lastWorkerInfo *types.WorkerData // Channels for communication outgoing chan *worker_pb.WorkerMessage @@ -404,7 +404,7 @@ func (c *GrpcAdminClient) handleIncomingWithReady(ready chan struct{}) { } // RegisterWorker registers the worker with the admin server -func (c *GrpcAdminClient) RegisterWorker(worker *types.Worker) error { +func (c *GrpcAdminClient) RegisterWorker(worker *types.WorkerData) error { // Store worker info for re-registration after reconnection c.mutex.Lock() c.lastWorkerInfo = worker @@ -420,7 +420,7 @@ func (c *GrpcAdminClient) RegisterWorker(worker *types.Worker) error { } // sendRegistration sends the registration message and waits for response -func (c *GrpcAdminClient) sendRegistration(worker *types.Worker) error { +func (c *GrpcAdminClient) sendRegistration(worker *types.WorkerData) error { capabilities := make([]string, len(worker.Capabilities)) for i, cap := range worker.Capabilities { capabilities[i] = string(cap) @@ -467,7 +467,7 @@ func (c *GrpcAdminClient) sendRegistration(worker *types.Worker) error { } // sendRegistrationSync sends the registration message synchronously -func (c *GrpcAdminClient) sendRegistrationSync(worker *types.Worker) error { +func (c *GrpcAdminClient) sendRegistrationSync(worker *types.WorkerData) error { capabilities := make([]string, len(worker.Capabilities)) for i, cap := range worker.Capabilities { capabilities[i] = string(cap) @@ -585,7 +585,7 @@ func (c *GrpcAdminClient) SendHeartbeat(workerID string, status *types.WorkerSta } // RequestTask requests a new task from admin server -func (c *GrpcAdminClient) RequestTask(workerID string, capabilities []types.TaskType) (*types.Task, error) { +func (c *GrpcAdminClient) RequestTask(workerID string, capabilities []types.TaskType) (*types.TaskInput, error) { if !c.connected { // If we're currently reconnecting, don't wait - just return no task c.mutex.RLock() @@ -646,7 +646,7 @@ func (c *GrpcAdminClient) RequestTask(workerID string, capabilities []types.Task workerID, taskAssign.TaskId, taskAssign.TaskType, taskAssign.Params.VolumeId) // Convert to our task type - task := &types.Task{ + task := &types.TaskInput{ ID: taskAssign.TaskId, Type: types.TaskType(taskAssign.TaskType), Status: types.TaskStatusAssigned, @@ -836,7 +836,7 @@ func (c *GrpcAdminClient) GetIncomingChannel() <-chan *worker_pb.AdminMessage { type MockAdminClient struct { workerID string connected bool - tasks []*types.Task + tasks []*types.TaskInput mutex sync.RWMutex } @@ -844,7 +844,7 @@ type MockAdminClient struct { func NewMockAdminClient() *MockAdminClient { return &MockAdminClient{ connected: true, - tasks: make([]*types.Task, 0), + tasks: make([]*types.TaskInput, 0), } } @@ -865,7 +865,7 @@ func (m *MockAdminClient) Disconnect() error { } // RegisterWorker mock implementation -func (m *MockAdminClient) RegisterWorker(worker *types.Worker) error { +func (m *MockAdminClient) RegisterWorker(worker *types.WorkerData) error { m.workerID = worker.ID glog.Infof("Mock: Worker %s registered with capabilities: %v", worker.ID, worker.Capabilities) return nil @@ -879,7 +879,7 @@ func (m *MockAdminClient) SendHeartbeat(workerID string, status *types.WorkerSta } // RequestTask mock implementation -func (m *MockAdminClient) RequestTask(workerID string, capabilities []types.TaskType) (*types.Task, error) { +func (m *MockAdminClient) RequestTask(workerID string, capabilities []types.TaskType) (*types.TaskInput, error) { m.mutex.Lock() defer m.mutex.Unlock() @@ -924,7 +924,7 @@ func (m *MockAdminClient) IsConnected() bool { } // AddMockTask adds a mock task for testing -func (m *MockAdminClient) AddMockTask(task *types.Task) { +func (m *MockAdminClient) AddMockTask(task *types.TaskInput) { m.mutex.Lock() defer m.mutex.Unlock() m.tasks = append(m.tasks, task) diff --git a/weed/worker/registry.go b/weed/worker/registry.go index e227beb6a..0b40ddec4 100644 --- a/weed/worker/registry.go +++ b/weed/worker/registry.go @@ -10,7 +10,7 @@ import ( // Registry manages workers and their statistics type Registry struct { - workers map[string]*types.Worker + workers map[string]*types.WorkerData stats *types.RegistryStats mutex sync.RWMutex } @@ -18,7 +18,7 @@ type Registry struct { // NewRegistry creates a new worker registry func NewRegistry() *Registry { return &Registry{ - workers: make(map[string]*types.Worker), + workers: make(map[string]*types.WorkerData), stats: &types.RegistryStats{ TotalWorkers: 0, ActiveWorkers: 0, @@ -33,7 +33,7 @@ func NewRegistry() *Registry { } // RegisterWorker registers a new worker -func (r *Registry) RegisterWorker(worker *types.Worker) error { +func (r *Registry) RegisterWorker(worker *types.WorkerData) error { r.mutex.Lock() defer r.mutex.Unlock() @@ -61,7 +61,7 @@ func (r *Registry) UnregisterWorker(workerID string) error { } // GetWorker returns a worker by ID -func (r *Registry) GetWorker(workerID string) (*types.Worker, bool) { +func (r *Registry) GetWorker(workerID string) (*types.WorkerData, bool) { r.mutex.RLock() defer r.mutex.RUnlock() @@ -70,11 +70,11 @@ func (r *Registry) GetWorker(workerID string) (*types.Worker, bool) { } // ListWorkers returns all registered workers -func (r *Registry) ListWorkers() []*types.Worker { +func (r *Registry) ListWorkers() []*types.WorkerData { r.mutex.RLock() defer r.mutex.RUnlock() - workers := make([]*types.Worker, 0, len(r.workers)) + workers := make([]*types.WorkerData, 0, len(r.workers)) for _, worker := range r.workers { workers = append(workers, worker) } @@ -82,11 +82,11 @@ func (r *Registry) ListWorkers() []*types.Worker { } // GetWorkersByCapability returns workers that support a specific capability -func (r *Registry) GetWorkersByCapability(capability types.TaskType) []*types.Worker { +func (r *Registry) GetWorkersByCapability(capability types.TaskType) []*types.WorkerData { r.mutex.RLock() defer r.mutex.RUnlock() - var workers []*types.Worker + var workers []*types.WorkerData for _, worker := range r.workers { for _, cap := range worker.Capabilities { if cap == capability { @@ -99,11 +99,11 @@ func (r *Registry) GetWorkersByCapability(capability types.TaskType) []*types.Wo } // GetAvailableWorkers returns workers that are available for new tasks -func (r *Registry) GetAvailableWorkers() []*types.Worker { +func (r *Registry) GetAvailableWorkers() []*types.WorkerData { r.mutex.RLock() defer r.mutex.RUnlock() - var workers []*types.Worker + var workers []*types.WorkerData for _, worker := range r.workers { if worker.Status == "active" && worker.CurrentLoad < worker.MaxConcurrent { workers = append(workers, worker) @@ -113,11 +113,11 @@ func (r *Registry) GetAvailableWorkers() []*types.Worker { } // GetBestWorkerForTask returns the best worker for a specific task -func (r *Registry) GetBestWorkerForTask(taskType types.TaskType) *types.Worker { +func (r *Registry) GetBestWorkerForTask(taskType types.TaskType) *types.WorkerData { r.mutex.RLock() defer r.mutex.RUnlock() - var bestWorker *types.Worker + var bestWorker *types.WorkerData var bestScore float64 for _, worker := range r.workers { @@ -277,11 +277,11 @@ func (r *Registry) GetTaskCapabilities() []types.TaskType { } // GetWorkersByStatus returns workers filtered by status -func (r *Registry) GetWorkersByStatus(status string) []*types.Worker { +func (r *Registry) GetWorkersByStatus(status string) []*types.WorkerData { r.mutex.RLock() defer r.mutex.RUnlock() - var workers []*types.Worker + var workers []*types.WorkerData for _, worker := range r.workers { if worker.Status == status { workers = append(workers, worker) diff --git a/weed/worker/tasks/balance/balance.go b/weed/worker/tasks/balance/balance.go deleted file mode 100644 index 0becb3415..000000000 --- a/weed/worker/tasks/balance/balance.go +++ /dev/null @@ -1,141 +0,0 @@ -package balance - -import ( - "context" - "fmt" - "time" - - "github.com/seaweedfs/seaweedfs/weed/glog" - "github.com/seaweedfs/seaweedfs/weed/worker/tasks" - "github.com/seaweedfs/seaweedfs/weed/worker/types" -) - -// Task implements balance operation to redistribute volumes across volume servers -type Task struct { - *tasks.BaseTask - server string - volumeID uint32 - collection string - - // Task parameters for accessing planned destinations - taskParams types.TaskParams -} - -// NewTask creates a new balance task instance -func NewTask(server string, volumeID uint32, collection string) *Task { - task := &Task{ - BaseTask: tasks.NewBaseTask(types.TaskTypeBalance), - server: server, - volumeID: volumeID, - collection: collection, - } - return task -} - -// Execute executes the balance task -func (t *Task) Execute(params types.TaskParams) error { - // Use BaseTask.ExecuteTask to handle logging initialization - return t.ExecuteTask(context.Background(), params, t.executeImpl) -} - -// executeImpl is the actual balance implementation -func (t *Task) executeImpl(ctx context.Context, params types.TaskParams) error { - // Store task parameters for accessing planned destinations - t.taskParams = params - - // Get planned destination - destNode := t.getPlannedDestination() - if destNode != "" { - t.LogWithFields("INFO", "Starting balance task with planned destination", map[string]interface{}{ - "volume_id": t.volumeID, - "source": t.server, - "destination": destNode, - "collection": t.collection, - }) - } else { - t.LogWithFields("INFO", "Starting balance task without specific destination", map[string]interface{}{ - "volume_id": t.volumeID, - "server": t.server, - "collection": t.collection, - }) - } - - // Simulate balance operation with progress updates - steps := []struct { - name string - duration time.Duration - progress float64 - }{ - {"Analyzing cluster state", 2 * time.Second, 15}, - {"Identifying optimal placement", 3 * time.Second, 35}, - {"Moving volume data", 6 * time.Second, 75}, - {"Updating cluster metadata", 2 * time.Second, 95}, - {"Verifying balance", 1 * time.Second, 100}, - } - - for _, step := range steps { - select { - case <-ctx.Done(): - t.LogWarning("Balance task cancelled during step: %s", step.name) - return ctx.Err() - default: - } - - if t.IsCancelled() { - t.LogWarning("Balance task cancelled by request during step: %s", step.name) - return fmt.Errorf("balance task cancelled") - } - - t.LogWithFields("INFO", "Executing balance step", map[string]interface{}{ - "step": step.name, - "progress": step.progress, - "duration": step.duration.String(), - "volume_id": t.volumeID, - }) - t.SetProgress(step.progress) - - // Simulate work - time.Sleep(step.duration) - } - - t.LogWithFields("INFO", "Balance task completed successfully", map[string]interface{}{ - "volume_id": t.volumeID, - "server": t.server, - "collection": t.collection, - "final_progress": 100.0, - }) - return nil -} - -// Validate validates the task parameters -func (t *Task) Validate(params types.TaskParams) error { - if params.VolumeID == 0 { - return fmt.Errorf("volume_id is required") - } - if params.Server == "" { - return fmt.Errorf("server is required") - } - return nil -} - -// getPlannedDestination extracts the planned destination node from task parameters -func (t *Task) getPlannedDestination() string { - if t.taskParams.TypedParams != nil { - if balanceParams := t.taskParams.TypedParams.GetBalanceParams(); balanceParams != nil { - if balanceParams.DestNode != "" { - glog.V(2).Infof("Found planned destination for volume %d: %s", t.volumeID, balanceParams.DestNode) - return balanceParams.DestNode - } - } - } - return "" -} - -// EstimateTime estimates the time needed for the task -func (t *Task) EstimateTime(params types.TaskParams) time.Duration { - // Base time for balance operation - baseTime := 35 * time.Second - - // Could adjust based on volume size or cluster state - return baseTime -} diff --git a/weed/worker/tasks/balance/balance_task.go b/weed/worker/tasks/balance/balance_task.go new file mode 100644 index 000000000..439a406a4 --- /dev/null +++ b/weed/worker/tasks/balance/balance_task.go @@ -0,0 +1,248 @@ +package balance + +import ( + "context" + "fmt" + "io" + "time" + + "github.com/seaweedfs/seaweedfs/weed/glog" + "github.com/seaweedfs/seaweedfs/weed/operation" + "github.com/seaweedfs/seaweedfs/weed/pb" + "github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" + "github.com/seaweedfs/seaweedfs/weed/storage/needle" + "github.com/seaweedfs/seaweedfs/weed/util" + "github.com/seaweedfs/seaweedfs/weed/worker/types" + "github.com/seaweedfs/seaweedfs/weed/worker/types/base" + "google.golang.org/grpc" +) + +// BalanceTask implements the Task interface +type BalanceTask struct { + *base.BaseTask + server string + volumeID uint32 + collection string + progress float64 +} + +// NewBalanceTask creates a new balance task instance +func NewBalanceTask(id string, server string, volumeID uint32, collection string) *BalanceTask { + return &BalanceTask{ + BaseTask: base.NewBaseTask(id, types.TaskTypeBalance), + server: server, + volumeID: volumeID, + collection: collection, + } +} + +// Execute implements the Task interface +func (t *BalanceTask) Execute(ctx context.Context, params *worker_pb.TaskParams) error { + if params == nil { + return fmt.Errorf("task parameters are required") + } + + balanceParams := params.GetBalanceParams() + if balanceParams == nil { + return fmt.Errorf("balance parameters are required") + } + + // Get planned destination + destNode := balanceParams.DestNode + + if destNode == "" { + return fmt.Errorf("destination node is required for balance task") + } + + t.GetLogger().WithFields(map[string]interface{}{ + "volume_id": t.volumeID, + "source": t.server, + "destination": destNode, + "collection": t.collection, + }).Info("Starting balance task - moving volume") + + sourceServer := pb.ServerAddress(t.server) + targetServer := pb.ServerAddress(destNode) + volumeId := needle.VolumeId(t.volumeID) + + // Step 1: Mark volume readonly + t.ReportProgress(10.0) + t.GetLogger().Info("Marking volume readonly for move") + if err := t.markVolumeReadonly(sourceServer, volumeId); err != nil { + return fmt.Errorf("failed to mark volume readonly: %v", err) + } + + // Step 2: Copy volume to destination + t.ReportProgress(20.0) + t.GetLogger().Info("Copying volume to destination") + lastAppendAtNs, err := t.copyVolume(sourceServer, targetServer, volumeId) + if err != nil { + return fmt.Errorf("failed to copy volume: %v", err) + } + + // Step 3: Mount volume on target and mark it readonly + t.ReportProgress(60.0) + t.GetLogger().Info("Mounting volume on target server") + if err := t.mountVolume(targetServer, volumeId); err != nil { + return fmt.Errorf("failed to mount volume on target: %v", err) + } + + // Step 4: Tail for updates + t.ReportProgress(70.0) + t.GetLogger().Info("Syncing final updates") + if err := t.tailVolume(sourceServer, targetServer, volumeId, lastAppendAtNs); err != nil { + glog.Warningf("Tail operation failed (may be normal): %v", err) + } + + // Step 5: Unmount from source + t.ReportProgress(85.0) + t.GetLogger().Info("Unmounting volume from source server") + if err := t.unmountVolume(sourceServer, volumeId); err != nil { + return fmt.Errorf("failed to unmount volume from source: %v", err) + } + + // Step 6: Delete from source + t.ReportProgress(95.0) + t.GetLogger().Info("Deleting volume from source server") + if err := t.deleteVolume(sourceServer, volumeId); err != nil { + return fmt.Errorf("failed to delete volume from source: %v", err) + } + + t.ReportProgress(100.0) + glog.Infof("Balance task completed successfully: volume %d moved from %s to %s", + t.volumeID, t.server, destNode) + return nil +} + +// Validate implements the UnifiedTask interface +func (t *BalanceTask) Validate(params *worker_pb.TaskParams) error { + if params == nil { + return fmt.Errorf("task parameters are required") + } + + balanceParams := params.GetBalanceParams() + if balanceParams == nil { + return fmt.Errorf("balance parameters are required") + } + + if params.VolumeId != t.volumeID { + return fmt.Errorf("volume ID mismatch: expected %d, got %d", t.volumeID, params.VolumeId) + } + + if params.Server != t.server { + return fmt.Errorf("source server mismatch: expected %s, got %s", t.server, params.Server) + } + + return nil +} + +// EstimateTime implements the UnifiedTask interface +func (t *BalanceTask) EstimateTime(params *worker_pb.TaskParams) time.Duration { + // Basic estimate based on simulated steps + return 14 * time.Second // Sum of all step durations +} + +// GetProgress returns current progress +func (t *BalanceTask) GetProgress() float64 { + return t.progress +} + +// Helper methods for real balance operations + +// markVolumeReadonly marks the volume readonly +func (t *BalanceTask) markVolumeReadonly(server pb.ServerAddress, volumeId needle.VolumeId) error { + return operation.WithVolumeServerClient(false, server, grpc.WithInsecure(), + func(client volume_server_pb.VolumeServerClient) error { + _, err := client.VolumeMarkReadonly(context.Background(), &volume_server_pb.VolumeMarkReadonlyRequest{ + VolumeId: uint32(volumeId), + }) + return err + }) +} + +// copyVolume copies volume from source to target server +func (t *BalanceTask) copyVolume(sourceServer, targetServer pb.ServerAddress, volumeId needle.VolumeId) (uint64, error) { + var lastAppendAtNs uint64 + + err := operation.WithVolumeServerClient(true, targetServer, grpc.WithInsecure(), + func(client volume_server_pb.VolumeServerClient) error { + stream, err := client.VolumeCopy(context.Background(), &volume_server_pb.VolumeCopyRequest{ + VolumeId: uint32(volumeId), + SourceDataNode: string(sourceServer), + }) + if err != nil { + return err + } + + for { + resp, recvErr := stream.Recv() + if recvErr != nil { + if recvErr == io.EOF { + break + } + return recvErr + } + + if resp.LastAppendAtNs != 0 { + lastAppendAtNs = resp.LastAppendAtNs + } else { + // Report copy progress + glog.V(1).Infof("Volume %d copy progress: %s", volumeId, + util.BytesToHumanReadable(uint64(resp.ProcessedBytes))) + } + } + + return nil + }) + + return lastAppendAtNs, err +} + +// mountVolume mounts the volume on the target server +func (t *BalanceTask) mountVolume(server pb.ServerAddress, volumeId needle.VolumeId) error { + return operation.WithVolumeServerClient(false, server, grpc.WithInsecure(), + func(client volume_server_pb.VolumeServerClient) error { + _, err := client.VolumeMount(context.Background(), &volume_server_pb.VolumeMountRequest{ + VolumeId: uint32(volumeId), + }) + return err + }) +} + +// tailVolume syncs remaining updates from source to target +func (t *BalanceTask) tailVolume(sourceServer, targetServer pb.ServerAddress, volumeId needle.VolumeId, sinceNs uint64) error { + return operation.WithVolumeServerClient(true, targetServer, grpc.WithInsecure(), + func(client volume_server_pb.VolumeServerClient) error { + _, err := client.VolumeTailReceiver(context.Background(), &volume_server_pb.VolumeTailReceiverRequest{ + VolumeId: uint32(volumeId), + SinceNs: sinceNs, + IdleTimeoutSeconds: 60, // 1 minute timeout + SourceVolumeServer: string(sourceServer), + }) + return err + }) +} + +// unmountVolume unmounts the volume from the server +func (t *BalanceTask) unmountVolume(server pb.ServerAddress, volumeId needle.VolumeId) error { + return operation.WithVolumeServerClient(false, server, grpc.WithInsecure(), + func(client volume_server_pb.VolumeServerClient) error { + _, err := client.VolumeUnmount(context.Background(), &volume_server_pb.VolumeUnmountRequest{ + VolumeId: uint32(volumeId), + }) + return err + }) +} + +// deleteVolume deletes the volume from the server +func (t *BalanceTask) deleteVolume(server pb.ServerAddress, volumeId needle.VolumeId) error { + return operation.WithVolumeServerClient(false, server, grpc.WithInsecure(), + func(client volume_server_pb.VolumeServerClient) error { + _, err := client.VolumeDelete(context.Background(), &volume_server_pb.VolumeDeleteRequest{ + VolumeId: uint32(volumeId), + OnlyEmpty: false, + }) + return err + }) +} diff --git a/weed/worker/tasks/balance/detection.go b/weed/worker/tasks/balance/detection.go index f4bcf3ca3..102f532a8 100644 --- a/weed/worker/tasks/balance/detection.go +++ b/weed/worker/tasks/balance/detection.go @@ -4,7 +4,9 @@ import ( "fmt" "time" + "github.com/seaweedfs/seaweedfs/weed/admin/topology" "github.com/seaweedfs/seaweedfs/weed/glog" + "github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" "github.com/seaweedfs/seaweedfs/weed/worker/tasks/base" "github.com/seaweedfs/seaweedfs/weed/worker/types" ) @@ -89,46 +91,144 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI Priority: types.TaskPriorityNormal, Reason: reason, ScheduleAt: time.Now(), - // TypedParams will be populated by the maintenance integration - // with destination planning information + } + + // Plan destination if ActiveTopology is available + if clusterInfo.ActiveTopology != nil { + destinationPlan, err := planBalanceDestination(clusterInfo.ActiveTopology, selectedVolume) + if err != nil { + glog.Warningf("Failed to plan balance destination for volume %d: %v", selectedVolume.VolumeID, err) + return nil, nil // Skip this task if destination planning fails + } + + // Create typed parameters with destination information + task.TypedParams = &worker_pb.TaskParams{ + VolumeId: selectedVolume.VolumeID, + Server: selectedVolume.Server, + Collection: selectedVolume.Collection, + VolumeSize: selectedVolume.Size, // Store original volume size for tracking changes + TaskParams: &worker_pb.TaskParams_BalanceParams{ + BalanceParams: &worker_pb.BalanceTaskParams{ + DestNode: destinationPlan.TargetNode, + EstimatedSize: destinationPlan.ExpectedSize, + PlacementScore: destinationPlan.PlacementScore, + PlacementConflicts: destinationPlan.Conflicts, + ForceMove: false, + TimeoutSeconds: 600, // 10 minutes default + }, + }, + } + + glog.V(1).Infof("Planned balance destination for volume %d: %s -> %s (score: %.2f)", + selectedVolume.VolumeID, selectedVolume.Server, destinationPlan.TargetNode, destinationPlan.PlacementScore) + } else { + glog.Warningf("No ActiveTopology available for destination planning in balance detection") + return nil, nil } return []*types.TaskDetectionResult{task}, nil } -// Scheduling implements the scheduling logic for balance tasks -func Scheduling(task *types.Task, runningTasks []*types.Task, availableWorkers []*types.Worker, config base.TaskConfig) bool { - balanceConfig := config.(*Config) - - // Count running balance tasks - runningBalanceCount := 0 - for _, runningTask := range runningTasks { - if runningTask.Type == types.TaskTypeBalance { - runningBalanceCount++ +// planBalanceDestination plans the destination for a balance operation +// This function implements destination planning logic directly in the detection phase +func planBalanceDestination(activeTopology *topology.ActiveTopology, selectedVolume *types.VolumeHealthMetrics) (*topology.DestinationPlan, error) { + // Get source node information from topology + var sourceRack, sourceDC string + + // 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 == selectedVolume.Server { + sourceDC = dc.Id + sourceRack = rack.Id + break + } + } + if sourceRack != "" { + break + } + } + if sourceDC != "" { + break + } } } - // Check concurrency limit - if runningBalanceCount >= balanceConfig.MaxConcurrent { - return false + // Get available disks, excluding the source node + availableDisks := activeTopology.GetAvailableDisks(topology.TaskTypeBalance, selectedVolume.Server) + if len(availableDisks) == 0 { + return nil, fmt.Errorf("no available disks for balance operation") } - // Check if we have available workers - availableWorkerCount := 0 - for _, worker := range availableWorkers { - for _, capability := range worker.Capabilities { - if capability == types.TaskTypeBalance { - availableWorkerCount++ - break - } + // Find the best destination disk based on balance criteria + var bestDisk *topology.DiskInfo + bestScore := -1.0 + + for _, disk := range availableDisks { + score := calculateBalanceScore(disk, sourceRack, sourceDC, selectedVolume.Size) + if score > bestScore { + bestScore = score + bestDisk = disk } } - return availableWorkerCount > 0 + if bestDisk == nil { + return nil, fmt.Errorf("no suitable destination found for balance operation") + } + + return &topology.DestinationPlan{ + TargetNode: bestDisk.NodeID, + TargetDisk: bestDisk.DiskID, + TargetRack: bestDisk.Rack, + TargetDC: bestDisk.DataCenter, + ExpectedSize: selectedVolume.Size, + PlacementScore: bestScore, + Conflicts: checkPlacementConflicts(bestDisk, sourceRack, sourceDC), + }, nil +} + +// calculateBalanceScore calculates placement score for balance operations +func calculateBalanceScore(disk *topology.DiskInfo, sourceRack, sourceDC string, volumeSize uint64) float64 { + if disk.DiskInfo == nil { + return 0.0 + } + + score := 0.0 + + // Prefer disks with lower current volume count (better for balance) + if disk.DiskInfo.MaxVolumeCount > 0 { + utilization := float64(disk.DiskInfo.VolumeCount) / float64(disk.DiskInfo.MaxVolumeCount) + score += (1.0 - utilization) * 40.0 // Up to 40 points for low utilization + } + + // Prefer different racks for better distribution + if disk.Rack != sourceRack { + score += 30.0 + } + + // Prefer different data centers for better distribution + if disk.DataCenter != sourceDC { + score += 20.0 + } + + // Prefer disks with lower current load + score += (10.0 - float64(disk.LoadCount)) // Up to 10 points for low load + + return score } -// CreateTask creates a new balance task instance -func CreateTask(params types.TaskParams) (types.TaskInterface, error) { - // Create and return the balance task using existing Task type - return NewTask(params.Server, params.VolumeID, params.Collection), nil +// checkPlacementConflicts checks for placement rule conflicts +func checkPlacementConflicts(disk *topology.DiskInfo, sourceRack, sourceDC string) []string { + var conflicts []string + + // For now, implement basic conflict detection + // This could be extended with more sophisticated placement rules + if disk.Rack == sourceRack && disk.DataCenter == sourceDC { + conflicts = append(conflicts, "same_rack_as_source") + } + + return conflicts } diff --git a/weed/worker/tasks/balance/balance_typed.go b/weed/worker/tasks/balance/execution.go similarity index 100% rename from weed/worker/tasks/balance/balance_typed.go rename to weed/worker/tasks/balance/execution.go diff --git a/weed/worker/tasks/balance/monitoring.go b/weed/worker/tasks/balance/monitoring.go new file mode 100644 index 000000000..517de2484 --- /dev/null +++ b/weed/worker/tasks/balance/monitoring.go @@ -0,0 +1,138 @@ +package balance + +import ( + "sync" + "time" +) + +// BalanceMetrics contains balance-specific monitoring data +type BalanceMetrics struct { + // Execution metrics + VolumesBalanced int64 `json:"volumes_balanced"` + TotalDataTransferred int64 `json:"total_data_transferred"` + AverageImbalance float64 `json:"average_imbalance"` + LastBalanceTime time.Time `json:"last_balance_time"` + + // Performance metrics + AverageTransferSpeed float64 `json:"average_transfer_speed_mbps"` + TotalExecutionTime int64 `json:"total_execution_time_seconds"` + SuccessfulOperations int64 `json:"successful_operations"` + FailedOperations int64 `json:"failed_operations"` + + // Current task metrics + CurrentImbalanceScore float64 `json:"current_imbalance_score"` + PlannedDestinations int `json:"planned_destinations"` + + mutex sync.RWMutex +} + +// NewBalanceMetrics creates a new balance metrics instance +func NewBalanceMetrics() *BalanceMetrics { + return &BalanceMetrics{ + LastBalanceTime: time.Now(), + } +} + +// RecordVolumeBalanced records a successful volume balance operation +func (m *BalanceMetrics) RecordVolumeBalanced(volumeSize int64, transferTime time.Duration) { + m.mutex.Lock() + defer m.mutex.Unlock() + + m.VolumesBalanced++ + m.TotalDataTransferred += volumeSize + m.SuccessfulOperations++ + m.LastBalanceTime = time.Now() + m.TotalExecutionTime += int64(transferTime.Seconds()) + + // Calculate average transfer speed (MB/s) + if transferTime > 0 { + speedMBps := float64(volumeSize) / (1024 * 1024) / transferTime.Seconds() + if m.AverageTransferSpeed == 0 { + m.AverageTransferSpeed = speedMBps + } else { + // Exponential moving average + m.AverageTransferSpeed = 0.8*m.AverageTransferSpeed + 0.2*speedMBps + } + } +} + +// RecordFailure records a failed balance operation +func (m *BalanceMetrics) RecordFailure() { + m.mutex.Lock() + defer m.mutex.Unlock() + + m.FailedOperations++ +} + +// UpdateImbalanceScore updates the current cluster imbalance score +func (m *BalanceMetrics) UpdateImbalanceScore(score float64) { + m.mutex.Lock() + defer m.mutex.Unlock() + + m.CurrentImbalanceScore = score + + // Update average imbalance with exponential moving average + if m.AverageImbalance == 0 { + m.AverageImbalance = score + } else { + m.AverageImbalance = 0.9*m.AverageImbalance + 0.1*score + } +} + +// SetPlannedDestinations sets the number of planned destinations +func (m *BalanceMetrics) SetPlannedDestinations(count int) { + m.mutex.Lock() + defer m.mutex.Unlock() + + m.PlannedDestinations = count +} + +// GetMetrics returns a copy of the current metrics (without the mutex) +func (m *BalanceMetrics) GetMetrics() BalanceMetrics { + m.mutex.RLock() + defer m.mutex.RUnlock() + + // Create a copy without the mutex to avoid copying lock value + return BalanceMetrics{ + VolumesBalanced: m.VolumesBalanced, + TotalDataTransferred: m.TotalDataTransferred, + AverageImbalance: m.AverageImbalance, + LastBalanceTime: m.LastBalanceTime, + AverageTransferSpeed: m.AverageTransferSpeed, + TotalExecutionTime: m.TotalExecutionTime, + SuccessfulOperations: m.SuccessfulOperations, + FailedOperations: m.FailedOperations, + CurrentImbalanceScore: m.CurrentImbalanceScore, + PlannedDestinations: m.PlannedDestinations, + } +} + +// GetSuccessRate returns the success rate as a percentage +func (m *BalanceMetrics) GetSuccessRate() float64 { + m.mutex.RLock() + defer m.mutex.RUnlock() + + total := m.SuccessfulOperations + m.FailedOperations + if total == 0 { + return 100.0 + } + return float64(m.SuccessfulOperations) / float64(total) * 100.0 +} + +// Reset resets all metrics to zero +func (m *BalanceMetrics) Reset() { + m.mutex.Lock() + defer m.mutex.Unlock() + + *m = BalanceMetrics{ + LastBalanceTime: time.Now(), + } +} + +// Global metrics instance for balance tasks +var globalBalanceMetrics = NewBalanceMetrics() + +// GetGlobalBalanceMetrics returns the global balance metrics instance +func GetGlobalBalanceMetrics() *BalanceMetrics { + return globalBalanceMetrics +} diff --git a/weed/worker/tasks/balance/balance_register.go b/weed/worker/tasks/balance/register.go similarity index 82% rename from weed/worker/tasks/balance/balance_register.go rename to weed/worker/tasks/balance/register.go index b26a40782..adf30c11c 100644 --- a/weed/worker/tasks/balance/balance_register.go +++ b/weed/worker/tasks/balance/register.go @@ -5,6 +5,7 @@ import ( "time" "github.com/seaweedfs/seaweedfs/weed/glog" + "github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" "github.com/seaweedfs/seaweedfs/weed/worker/tasks" "github.com/seaweedfs/seaweedfs/weed/worker/tasks/base" "github.com/seaweedfs/seaweedfs/weed/worker/types" @@ -35,9 +36,19 @@ func RegisterBalanceTask() { Icon: "fas fa-balance-scale text-warning", Capabilities: []string{"balance", "distribution"}, - Config: config, - ConfigSpec: GetConfigSpec(), - CreateTask: CreateTask, + Config: config, + ConfigSpec: GetConfigSpec(), + CreateTask: func(params *worker_pb.TaskParams) (types.Task, error) { + if params == nil { + return nil, fmt.Errorf("task parameters are required") + } + return NewBalanceTask( + fmt.Sprintf("balance-%d", params.VolumeId), + params.Server, + params.VolumeId, + params.Collection, + ), nil + }, DetectionFunc: Detection, ScanInterval: 30 * time.Minute, SchedulingFunc: Scheduling, diff --git a/weed/worker/tasks/balance/scheduling.go b/weed/worker/tasks/balance/scheduling.go new file mode 100644 index 000000000..878686309 --- /dev/null +++ b/weed/worker/tasks/balance/scheduling.go @@ -0,0 +1,37 @@ +package balance + +import ( + "github.com/seaweedfs/seaweedfs/weed/worker/tasks/base" + "github.com/seaweedfs/seaweedfs/weed/worker/types" +) + +// Scheduling implements the scheduling logic for balance tasks +func Scheduling(task *types.TaskInput, runningTasks []*types.TaskInput, availableWorkers []*types.WorkerData, config base.TaskConfig) bool { + balanceConfig := config.(*Config) + + // Count running balance tasks + runningBalanceCount := 0 + for _, runningTask := range runningTasks { + if runningTask.Type == types.TaskTypeBalance { + runningBalanceCount++ + } + } + + // Check concurrency limit + if runningBalanceCount >= balanceConfig.MaxConcurrent { + return false + } + + // Check if we have available workers + availableWorkerCount := 0 + for _, worker := range availableWorkers { + for _, capability := range worker.Capabilities { + if capability == types.TaskTypeBalance { + availableWorkerCount++ + break + } + } + } + + return availableWorkerCount > 0 +} diff --git a/weed/worker/tasks/base/generic_components.go b/weed/worker/tasks/base/generic_components.go index 27ad1bb29..0a41bbd76 100644 --- a/weed/worker/tasks/base/generic_components.go +++ b/weed/worker/tasks/base/generic_components.go @@ -58,7 +58,7 @@ func (s *GenericScheduler) GetTaskType() types.TaskType { } // CanScheduleNow determines if a task can be scheduled using the task definition's function -func (s *GenericScheduler) CanScheduleNow(task *types.Task, runningTasks []*types.Task, availableWorkers []*types.Worker) bool { +func (s *GenericScheduler) CanScheduleNow(task *types.TaskInput, runningTasks []*types.TaskInput, availableWorkers []*types.WorkerData) bool { if s.taskDef.SchedulingFunc == nil { return s.defaultCanSchedule(task, runningTasks, availableWorkers) } @@ -66,7 +66,7 @@ func (s *GenericScheduler) CanScheduleNow(task *types.Task, runningTasks []*type } // defaultCanSchedule provides default scheduling logic -func (s *GenericScheduler) defaultCanSchedule(task *types.Task, runningTasks []*types.Task, availableWorkers []*types.Worker) bool { +func (s *GenericScheduler) defaultCanSchedule(task *types.TaskInput, runningTasks []*types.TaskInput, availableWorkers []*types.WorkerData) bool { if !s.taskDef.Config.IsEnabled() { return false } @@ -103,7 +103,7 @@ func (s *GenericScheduler) defaultCanSchedule(task *types.Task, runningTasks []* } // GetPriority returns the priority for this task -func (s *GenericScheduler) GetPriority(task *types.Task) types.TaskPriority { +func (s *GenericScheduler) GetPriority(task *types.TaskInput) types.TaskPriority { return task.Priority } diff --git a/weed/worker/tasks/base/registration.go b/weed/worker/tasks/base/registration.go index 416b6f6b8..bef96d291 100644 --- a/weed/worker/tasks/base/registration.go +++ b/weed/worker/tasks/base/registration.go @@ -29,13 +29,28 @@ func NewGenericFactory(taskDef *TaskDefinition) *GenericFactory { } // Create creates a task instance using the task definition -func (f *GenericFactory) Create(params types.TaskParams) (types.TaskInterface, error) { +func (f *GenericFactory) Create(params *worker_pb.TaskParams) (types.Task, error) { if f.taskDef.CreateTask == nil { return nil, fmt.Errorf("no task creation function defined for %s", f.taskDef.Type) } return f.taskDef.CreateTask(params) } +// Type returns the task type +func (f *GenericFactory) Type() string { + return string(f.taskDef.Type) +} + +// Description returns a description of what this task does +func (f *GenericFactory) Description() string { + return f.taskDef.Description +} + +// Capabilities returns the task capabilities +func (f *GenericFactory) Capabilities() []string { + return f.taskDef.Capabilities +} + // GenericSchemaProvider provides config schema from TaskDefinition type GenericSchemaProvider struct { taskDef *TaskDefinition @@ -149,7 +164,8 @@ func validateTaskDefinition(taskDef *TaskDefinition) error { if taskDef.Config == nil { return fmt.Errorf("task config is required") } - // CreateTask is optional for tasks that use the typed task system - // The typed system registers tasks separately via types.RegisterGlobalTypedTask() + if taskDef.CreateTask == nil { + return fmt.Errorf("task creation function is required") + } return nil } diff --git a/weed/worker/tasks/base/task_definition.go b/weed/worker/tasks/base/task_definition.go index 6689d9c81..5ebc2a4b6 100644 --- a/weed/worker/tasks/base/task_definition.go +++ b/weed/worker/tasks/base/task_definition.go @@ -26,14 +26,14 @@ type TaskDefinition struct { ConfigSpec ConfigSpec // Task creation - CreateTask func(params types.TaskParams) (types.TaskInterface, error) + CreateTask func(params *worker_pb.TaskParams) (types.Task, error) // Detection logic DetectionFunc func(metrics []*types.VolumeHealthMetrics, info *types.ClusterInfo, config TaskConfig) ([]*types.TaskDetectionResult, error) ScanInterval time.Duration // Scheduling logic - SchedulingFunc func(task *types.Task, running []*types.Task, workers []*types.Worker, config TaskConfig) bool + SchedulingFunc func(task *types.TaskInput, running []*types.TaskInput, workers []*types.WorkerData, config TaskConfig) bool MaxConcurrent int RepeatInterval time.Duration } diff --git a/weed/worker/tasks/erasure_coding/detection.go b/weed/worker/tasks/erasure_coding/detection.go index 450080a12..1122d2721 100644 --- a/weed/worker/tasks/erasure_coding/detection.go +++ b/weed/worker/tasks/erasure_coding/detection.go @@ -5,7 +5,10 @@ import ( "strings" "time" + "github.com/seaweedfs/seaweedfs/weed/admin/topology" "github.com/seaweedfs/seaweedfs/weed/glog" + "github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" + "github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding" "github.com/seaweedfs/seaweedfs/weed/worker/tasks/base" "github.com/seaweedfs/seaweedfs/weed/worker/types" ) @@ -69,6 +72,38 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI float64(metric.Size)/(1024*1024), ecConfig.MinSizeMB), ScheduleAt: now, } + + // Plan EC destinations if ActiveTopology is available + if clusterInfo.ActiveTopology != nil { + 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 + } + + // Find all volume replicas from topology + replicas := findVolumeReplicas(clusterInfo.ActiveTopology, metric.VolumeID, metric.Collection) + glog.V(1).Infof("Found %d replicas for volume %d: %v", len(replicas), metric.VolumeID, replicas) + + // Create typed parameters with EC destination information and replicas + result.TypedParams = &worker_pb.TaskParams{ + VolumeId: metric.VolumeID, + Server: metric.Server, + Collection: metric.Collection, + VolumeSize: metric.Size, // Store original volume size for tracking changes + Replicas: replicas, // Include all volume replicas for deletion + 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 + } + results = append(results, result) } else { // Count debug reasons @@ -105,36 +140,277 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI return results, nil } -// Scheduling implements the scheduling logic for erasure coding tasks -func Scheduling(task *types.Task, runningTasks []*types.Task, availableWorkers []*types.Worker, config base.TaskConfig) bool { - ecConfig := config.(*Config) +// 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) { + // Get source node information from topology + var sourceRack, sourceDC string - // Check if we have available workers - if len(availableWorkers) == 0 { - return false + // 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 + } + } + } + + // Determine minimum shard disk locations based on configuration + minTotalDisks := 4 + + // Get available disks for EC placement (include source node for EC) + availableDisks := activeTopology.GetAvailableDisks(topology.TaskTypeErasureCoding, "") + if len(availableDisks) < minTotalDisks { + return nil, fmt.Errorf("insufficient disks for EC placement: need %d, have %d", minTotalDisks, len(availableDisks)) + } + + // Select best disks for EC placement with rack/DC diversity + selectedDisks := selectBestECDestinations(availableDisks, sourceRack, sourceDC, erasure_coding.TotalShardsCount) + if len(selectedDisks) < minTotalDisks { + return nil, fmt.Errorf("found %d disks, but could not find %d suitable destinations for EC placement", len(selectedDisks), minTotalDisks) + } + + 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, + ExpectedSize: 0, // EC shards don't have predetermined size + PlacementScore: calculateECScore(disk, sourceRack, sourceDC), + Conflicts: checkECPlacementConflicts(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]++ + } + + return &topology.MultiDestinationPlan{ + Plans: plans, + TotalShards: len(plans), + SuccessfulRack: len(rackCount), + SuccessfulDCs: len(dcCount), + }, nil +} + +// createECTaskParams creates EC task parameters from the multi-destination plan +func createECTaskParams(multiPlan *topology.MultiDestinationPlan) *worker_pb.ErasureCodingTaskParams { + var destinations []*worker_pb.ECDestination + + for _, plan := range multiPlan.Plans { + destination := &worker_pb.ECDestination{ + Node: plan.TargetNode, + DiskId: plan.TargetDisk, + Rack: plan.TargetRack, + DataCenter: plan.TargetDC, + PlacementScore: plan.PlacementScore, + } + destinations = append(destinations, destination) + } + + // Collect placement conflicts from all destinations + var placementConflicts []string + for _, plan := range multiPlan.Plans { + placementConflicts = append(placementConflicts, plan.Conflicts...) + } + + return &worker_pb.ErasureCodingTaskParams{ + Destinations: destinations, + DataShards: erasure_coding.DataShardsCount, // Standard data shards + ParityShards: erasure_coding.ParityShardsCount, // Standard parity shards + PlacementConflicts: placementConflicts, + } +} + +// 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 } - // Count running EC tasks - runningCount := 0 - for _, runningTask := range runningTasks { - if runningTask.Type == types.TaskTypeErasureCoding { - runningCount++ + 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 } } - // Check concurrency limit - if runningCount >= ecConfig.MaxConcurrent { + 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 + + // Prefer disks with available capacity + if disk.DiskInfo.MaxVolumeCount > 0 { + utilization := float64(disk.DiskInfo.VolumeCount) / float64(disk.DiskInfo.MaxVolumeCount) + score += (1.0 - utilization) * 50.0 // Up to 50 points for available capacity + } + + // Prefer different racks for better distribution + if disk.Rack != sourceRack { + score += 30.0 + } + + // Prefer different data centers for better distribution + if disk.DataCenter != sourceDC { + score += 20.0 + } + + // Consider current load + score += (10.0 - float64(disk.LoadCount)) // Up to 10 points for low load + + return score +} + +// isDiskSuitableForEC checks if a disk is suitable for EC placement +func isDiskSuitableForEC(disk *topology.DiskInfo) bool { + if disk.DiskInfo == nil { + return false + } + + // Check if disk has capacity + if disk.DiskInfo.VolumeCount >= disk.DiskInfo.MaxVolumeCount { + return false + } + + // Check if disk is not overloaded + if disk.LoadCount > 10 { // Arbitrary threshold return false } - // Check if any worker can handle EC tasks - for _, worker := range availableWorkers { - for _, capability := range worker.Capabilities { - if capability == types.TaskTypeErasureCoding { - return true + return true +} + +// checkECPlacementConflicts checks for placement rule conflicts in EC operations +func checkECPlacementConflicts(disk *topology.DiskInfo, sourceRack, sourceDC string) []string { + var conflicts []string + + // For EC, being on the same rack as source is often acceptable + // but we note it as potential conflict for monitoring + if disk.Rack == sourceRack && disk.DataCenter == sourceDC { + conflicts = append(conflicts, "same_rack_as_source") + } + + return conflicts +} + +// 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 + } + } + } } } } - return false + return replicaServers } diff --git a/weed/worker/tasks/erasure_coding/ec.go b/weed/worker/tasks/erasure_coding/ec.go deleted file mode 100644 index 8dc7a1cd0..000000000 --- a/weed/worker/tasks/erasure_coding/ec.go +++ /dev/null @@ -1,785 +0,0 @@ -package erasure_coding - -import ( - "context" - "fmt" - "io" - "math" - "os" - "path/filepath" - "strings" - "sync" - "time" - - "github.com/seaweedfs/seaweedfs/weed/glog" - "github.com/seaweedfs/seaweedfs/weed/operation" - "github.com/seaweedfs/seaweedfs/weed/pb" - "github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb" - "github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" - "github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding" - "github.com/seaweedfs/seaweedfs/weed/storage/needle" - "github.com/seaweedfs/seaweedfs/weed/storage/volume_info" - "github.com/seaweedfs/seaweedfs/weed/worker/tasks/base" - "github.com/seaweedfs/seaweedfs/weed/worker/types" - "google.golang.org/grpc" - "google.golang.org/grpc/credentials/insecure" -) - -// Task implements comprehensive erasure coding with protobuf parameters -type Task struct { - *base.BaseTypedTask - - // Current task state - sourceServer string - volumeID uint32 - collection string - workDir string - masterClient string - grpcDialOpt grpc.DialOption - - // EC parameters from protobuf - destinations []*worker_pb.ECDestination // Disk-aware destinations - existingShardLocations []*worker_pb.ExistingECShardLocation // Existing shards to cleanup - estimatedShardSize uint64 - dataShards int - parityShards int - cleanupSource bool - - // Progress tracking - currentStep string - stepProgress map[string]float64 -} - -// NewTask creates a new erasure coding task -func NewTask() types.TypedTaskInterface { - task := &Task{ - BaseTypedTask: base.NewBaseTypedTask(types.TaskTypeErasureCoding), - masterClient: "localhost:9333", // Default master client - workDir: "/tmp/seaweedfs_ec_work", // Default work directory - grpcDialOpt: grpc.WithTransportCredentials(insecure.NewCredentials()), // Default to insecure - dataShards: erasure_coding.DataShardsCount, // Use package constant - parityShards: erasure_coding.ParityShardsCount, // Use package constant - stepProgress: make(map[string]float64), - } - return task -} - -// ValidateTyped validates the typed parameters for EC task -func (t *Task) ValidateTyped(params *worker_pb.TaskParams) error { - // Basic validation from base class - if err := t.BaseTypedTask.ValidateTyped(params); err != nil { - return err - } - - // Check that we have EC-specific parameters - ecParams := params.GetErasureCodingParams() - if ecParams == nil { - return fmt.Errorf("erasure_coding_params is required for EC task") - } - - // Require destinations - if len(ecParams.Destinations) == 0 { - return fmt.Errorf("destinations must be specified for EC task") - } - - // DataShards and ParityShards are constants from erasure_coding package - expectedDataShards := int32(erasure_coding.DataShardsCount) - expectedParityShards := int32(erasure_coding.ParityShardsCount) - - if ecParams.DataShards > 0 && ecParams.DataShards != expectedDataShards { - return fmt.Errorf("data_shards must be %d (fixed constant), got %d", expectedDataShards, ecParams.DataShards) - } - if ecParams.ParityShards > 0 && ecParams.ParityShards != expectedParityShards { - return fmt.Errorf("parity_shards must be %d (fixed constant), got %d", expectedParityShards, ecParams.ParityShards) - } - - // Validate destination count - destinationCount := len(ecParams.Destinations) - totalShards := expectedDataShards + expectedParityShards - if totalShards > int32(destinationCount) { - return fmt.Errorf("insufficient destinations: need %d, have %d", totalShards, destinationCount) - } - - return nil -} - -// EstimateTimeTyped estimates the time needed for EC processing based on protobuf parameters -func (t *Task) EstimateTimeTyped(params *worker_pb.TaskParams) time.Duration { - baseTime := 20 * time.Minute // Processing takes time due to comprehensive operations - - ecParams := params.GetErasureCodingParams() - if ecParams != nil && ecParams.EstimatedShardSize > 0 { - // More accurate estimate based on shard size - // Account for copying, encoding, and distribution - gbSize := ecParams.EstimatedShardSize / (1024 * 1024 * 1024) - estimatedTime := time.Duration(gbSize*2) * time.Minute // 2 minutes per GB - if estimatedTime > baseTime { - return estimatedTime - } - } - - return baseTime -} - -// ExecuteTyped implements the actual erasure coding workflow with typed parameters -func (t *Task) ExecuteTyped(params *worker_pb.TaskParams) error { - // Extract basic parameters - t.volumeID = params.VolumeId - t.sourceServer = params.Server - t.collection = params.Collection - - // Extract EC-specific parameters - ecParams := params.GetErasureCodingParams() - if ecParams != nil { - t.destinations = ecParams.Destinations // Store disk-aware destinations - t.existingShardLocations = ecParams.ExistingShardLocations // Store existing shards for cleanup - t.estimatedShardSize = ecParams.EstimatedShardSize - t.cleanupSource = ecParams.CleanupSource - - // DataShards and ParityShards are constants, don't override from parameters - // t.dataShards and t.parityShards are already set to constants in NewTask - - if ecParams.WorkingDir != "" { - t.workDir = ecParams.WorkingDir - } - if ecParams.MasterClient != "" { - t.masterClient = ecParams.MasterClient - } - } - - // Determine available destinations for logging - var availableDestinations []string - for _, dest := range t.destinations { - availableDestinations = append(availableDestinations, fmt.Sprintf("%s(disk:%d)", dest.Node, dest.DiskId)) - } - - glog.V(1).Infof("Starting EC task for volume %d: %s -> %v (data:%d, parity:%d)", - t.volumeID, t.sourceServer, availableDestinations, t.dataShards, t.parityShards) - - // Create unique working directory for this task - taskWorkDir := filepath.Join(t.workDir, fmt.Sprintf("vol_%d_%d", t.volumeID, time.Now().Unix())) - if err := os.MkdirAll(taskWorkDir, 0755); err != nil { - return fmt.Errorf("failed to create task working directory %s: %v", taskWorkDir, err) - } - glog.V(1).Infof("WORKFLOW: Created working directory: %s", taskWorkDir) - - // Ensure cleanup of working directory - defer func() { - if err := os.RemoveAll(taskWorkDir); err != nil { - glog.Warningf("Failed to cleanup working directory %s: %v", taskWorkDir, err) - } else { - glog.V(1).Infof("WORKFLOW: Cleaned up working directory: %s", taskWorkDir) - } - }() - - // Step 1: Collect volume locations from master - glog.V(1).Infof("WORKFLOW STEP 1: Collecting volume locations from master") - t.SetProgress(5.0) - volumeId := needle.VolumeId(t.volumeID) - volumeLocations, err := t.collectVolumeLocations(volumeId) - if err != nil { - return fmt.Errorf("failed to collect volume locations before EC encoding: %v", err) - } - glog.V(1).Infof("WORKFLOW: Found volume %d on %d servers: %v", t.volumeID, len(volumeLocations), volumeLocations) - - // Convert ServerAddress slice to string slice - var locationStrings []string - for _, addr := range volumeLocations { - locationStrings = append(locationStrings, string(addr)) - } - - // Step 2: Check if volume has sufficient size for EC encoding - if !t.shouldPerformECEncoding(locationStrings) { - glog.Infof("Volume %d does not meet EC encoding criteria, skipping", t.volumeID) - t.SetProgress(100.0) - return nil - } - - // Step 2A: Cleanup existing EC shards if any - glog.V(1).Infof("WORKFLOW STEP 2A: Cleaning up existing EC shards for volume %d", t.volumeID) - t.SetProgress(10.0) - err = t.cleanupExistingEcShards() - if err != nil { - glog.Warningf("Failed to cleanup existing EC shards (continuing anyway): %v", err) - // Don't fail the task - this is just cleanup - } - glog.V(1).Infof("WORKFLOW: Existing EC shards cleanup completed for volume %d", t.volumeID) - - // Step 3: Mark volume readonly on all servers - glog.V(1).Infof("WORKFLOW STEP 2B: Marking volume %d readonly on all replica servers", t.volumeID) - t.SetProgress(15.0) - err = t.markVolumeReadonlyOnAllReplicas(needle.VolumeId(t.volumeID), locationStrings) - if err != nil { - return fmt.Errorf("failed to mark volume readonly: %v", err) - } - glog.V(1).Infof("WORKFLOW: Volume %d marked readonly on all replicas", t.volumeID) - - // Step 5: Copy volume files (.dat, .idx) to EC worker - glog.V(1).Infof("WORKFLOW STEP 3: Copying volume files from source server %s to EC worker", t.sourceServer) - t.SetProgress(25.0) - localVolumeFiles, err := t.copyVolumeFilesToWorker(taskWorkDir) - if err != nil { - return fmt.Errorf("failed to copy volume files to EC worker: %v", err) - } - glog.V(1).Infof("WORKFLOW: Volume files copied to EC worker: %v", localVolumeFiles) - - // Step 6: Generate EC shards locally on EC worker - glog.V(1).Infof("WORKFLOW STEP 4: Generating EC shards locally on EC worker") - t.SetProgress(40.0) - localShardFiles, err := t.generateEcShardsLocally(localVolumeFiles, taskWorkDir) - if err != nil { - return fmt.Errorf("failed to generate EC shards locally: %v", err) - } - glog.V(1).Infof("WORKFLOW: EC shards generated locally: %d shard files", len(localShardFiles)) - - // Step 7: Distribute shards from EC worker to destination servers - glog.V(1).Infof("WORKFLOW STEP 5: Distributing EC shards from worker to destination servers") - t.SetProgress(60.0) - err = t.distributeEcShardsFromWorker(localShardFiles) - if err != nil { - return fmt.Errorf("failed to distribute EC shards from worker: %v", err) - } - glog.V(1).Infof("WORKFLOW: EC shards distributed to all destination servers") - - // Step 8: Mount EC shards on destination servers - glog.V(1).Infof("WORKFLOW STEP 6: Mounting EC shards on destination servers") - t.SetProgress(80.0) - err = t.mountEcShardsOnDestinations() - if err != nil { - return fmt.Errorf("failed to mount EC shards: %v", err) - } - glog.V(1).Infof("WORKFLOW: EC shards mounted successfully") - - // Step 9: Delete original volume from all locations - glog.V(1).Infof("WORKFLOW STEP 7: Deleting original volume %d from all replica servers", t.volumeID) - t.SetProgress(90.0) - err = t.deleteVolumeFromAllLocations(needle.VolumeId(t.volumeID), locationStrings) - if err != nil { - return fmt.Errorf("failed to delete original volume: %v", err) - } - glog.V(1).Infof("WORKFLOW: Original volume %d deleted from all locations", t.volumeID) - - t.SetProgress(100.0) - glog.Infof("EC task completed successfully for volume %d", t.volumeID) - return nil -} - -// collectVolumeLocations gets volume location from master (placeholder implementation) -func (t *Task) collectVolumeLocations(volumeId needle.VolumeId) ([]pb.ServerAddress, error) { - // For now, return a placeholder implementation - // Full implementation would call master to get volume locations - return []pb.ServerAddress{pb.ServerAddress(t.sourceServer)}, nil -} - -// cleanupExistingEcShards deletes existing EC shards using planned locations -func (t *Task) cleanupExistingEcShards() error { - if len(t.existingShardLocations) == 0 { - glog.V(1).Infof("No existing EC shards to cleanup for volume %d", t.volumeID) - return nil - } - - glog.V(1).Infof("Cleaning up existing EC shards for volume %d on %d servers", t.volumeID, len(t.existingShardLocations)) - - // Delete existing shards from each location using planned shard locations - for _, location := range t.existingShardLocations { - if len(location.ShardIds) == 0 { - continue - } - - glog.V(1).Infof("Deleting existing EC shards %v from %s for volume %d", location.ShardIds, location.Node, t.volumeID) - - err := operation.WithVolumeServerClient(false, pb.ServerAddress(location.Node), t.grpcDialOpt, - func(client volume_server_pb.VolumeServerClient) error { - _, deleteErr := client.VolumeEcShardsDelete(context.Background(), &volume_server_pb.VolumeEcShardsDeleteRequest{ - VolumeId: t.volumeID, - Collection: t.collection, - ShardIds: location.ShardIds, - }) - return deleteErr - }) - - if err != nil { - glog.Errorf("Failed to delete existing EC shards %v from %s for volume %d: %v", location.ShardIds, location.Node, t.volumeID, err) - // Continue with other servers - don't fail the entire cleanup - } else { - glog.V(1).Infof("Successfully deleted existing EC shards %v from %s for volume %d", location.ShardIds, location.Node, t.volumeID) - } - } - - glog.V(1).Infof("Completed cleanup of existing EC shards for volume %d", t.volumeID) - return nil -} - -// shouldPerformECEncoding checks if the volume meets criteria for EC encoding -func (t *Task) shouldPerformECEncoding(volumeLocations []string) bool { - // For now, always proceed with EC encoding if volume exists - // This can be extended with volume size checks, etc. - return len(volumeLocations) > 0 -} - -// markVolumeReadonlyOnAllReplicas marks the volume as readonly on all replica servers -func (t *Task) markVolumeReadonlyOnAllReplicas(volumeId needle.VolumeId, volumeLocations []string) error { - glog.V(1).Infof("Marking volume %d readonly on %d servers", volumeId, len(volumeLocations)) - - // Mark volume readonly on all replica servers - for _, location := range volumeLocations { - glog.V(1).Infof("Marking volume %d readonly on %s", volumeId, location) - - err := operation.WithVolumeServerClient(false, pb.ServerAddress(location), t.grpcDialOpt, - func(client volume_server_pb.VolumeServerClient) error { - _, markErr := client.VolumeMarkReadonly(context.Background(), &volume_server_pb.VolumeMarkReadonlyRequest{ - VolumeId: uint32(volumeId), - }) - return markErr - }) - - if err != nil { - glog.Errorf("Failed to mark volume %d readonly on %s: %v", volumeId, location, err) - return fmt.Errorf("failed to mark volume %d readonly on %s: %v", volumeId, location, err) - } - - glog.V(1).Infof("Successfully marked volume %d readonly on %s", volumeId, location) - } - - glog.V(1).Infof("Successfully marked volume %d readonly on all %d locations", volumeId, len(volumeLocations)) - return nil -} - -// copyVolumeFilesToWorker copies .dat and .idx files from source server to local worker -func (t *Task) copyVolumeFilesToWorker(workDir string) (map[string]string, error) { - localFiles := make(map[string]string) - - // Copy .dat file - datFile := fmt.Sprintf("%s.dat", filepath.Join(workDir, fmt.Sprintf("%d", t.volumeID))) - err := t.copyFileFromSource(".dat", datFile) - if err != nil { - return nil, fmt.Errorf("failed to copy .dat file: %v", err) - } - localFiles["dat"] = datFile - glog.V(1).Infof("Copied .dat file to: %s", datFile) - - // Copy .idx file - idxFile := fmt.Sprintf("%s.idx", filepath.Join(workDir, fmt.Sprintf("%d", t.volumeID))) - err = t.copyFileFromSource(".idx", idxFile) - if err != nil { - return nil, fmt.Errorf("failed to copy .idx file: %v", err) - } - localFiles["idx"] = idxFile - glog.V(1).Infof("Copied .idx file to: %s", idxFile) - - return localFiles, nil -} - -// copyFileFromSource copies a file from source server to local path using gRPC streaming -func (t *Task) copyFileFromSource(ext, localPath string) error { - return operation.WithVolumeServerClient(false, pb.ServerAddress(t.sourceServer), t.grpcDialOpt, - func(client volume_server_pb.VolumeServerClient) error { - stream, err := client.CopyFile(context.Background(), &volume_server_pb.CopyFileRequest{ - VolumeId: t.volumeID, - Collection: t.collection, - Ext: ext, - StopOffset: uint64(math.MaxInt64), - }) - if err != nil { - return fmt.Errorf("failed to initiate file copy: %v", err) - } - - // Create local file - localFile, err := os.Create(localPath) - if err != nil { - return fmt.Errorf("failed to create local file %s: %v", localPath, err) - } - defer localFile.Close() - - // Stream data and write to local file - totalBytes := int64(0) - for { - resp, err := stream.Recv() - if err == io.EOF { - break - } - if err != nil { - return fmt.Errorf("failed to receive file data: %v", err) - } - - if len(resp.FileContent) > 0 { - written, writeErr := localFile.Write(resp.FileContent) - if writeErr != nil { - return fmt.Errorf("failed to write to local file: %v", writeErr) - } - totalBytes += int64(written) - } - } - - glog.V(1).Infof("Successfully copied %s (%d bytes) from %s to %s", ext, totalBytes, t.sourceServer, localPath) - return nil - }) -} - -// generateEcShardsLocally generates EC shards from local volume files -func (t *Task) generateEcShardsLocally(localFiles map[string]string, workDir string) (map[string]string, error) { - datFile := localFiles["dat"] - idxFile := localFiles["idx"] - - if datFile == "" || idxFile == "" { - return nil, fmt.Errorf("missing required volume files: dat=%s, idx=%s", datFile, idxFile) - } - - // Get base name without extension for EC operations - baseName := strings.TrimSuffix(datFile, ".dat") - - shardFiles := make(map[string]string) - - glog.V(1).Infof("Generating EC shards from local files: dat=%s, idx=%s", datFile, idxFile) - - // Generate EC shard files (.ec00 ~ .ec13) - if err := erasure_coding.WriteEcFiles(baseName); err != nil { - return nil, fmt.Errorf("failed to generate EC shard files: %v", err) - } - - // Generate .ecx file from .idx - if err := erasure_coding.WriteSortedFileFromIdx(idxFile, ".ecx"); err != nil { - return nil, fmt.Errorf("failed to generate .ecx file: %v", err) - } - - // Collect generated shard file paths - for i := 0; i < erasure_coding.TotalShardsCount; i++ { - shardFile := fmt.Sprintf("%s.ec%02d", baseName, i) - if _, err := os.Stat(shardFile); err == nil { - shardFiles[fmt.Sprintf("ec%02d", i)] = shardFile - } - } - - // Add metadata files - ecxFile := idxFile + ".ecx" - if _, err := os.Stat(ecxFile); err == nil { - shardFiles["ecx"] = ecxFile - } - - // Generate .vif file (volume info) - vifFile := baseName + ".vif" - // Create basic volume info - in a real implementation, this would come from the original volume - volumeInfo := &volume_server_pb.VolumeInfo{ - Version: uint32(needle.GetCurrentVersion()), - } - if err := volume_info.SaveVolumeInfo(vifFile, volumeInfo); err != nil { - glog.Warningf("Failed to create .vif file: %v", err) - } else { - shardFiles["vif"] = vifFile - } - - glog.V(1).Infof("Generated %d EC files locally", len(shardFiles)) - return shardFiles, nil -} - -func (t *Task) copyEcShardsToDestinations() error { - if len(t.destinations) == 0 { - return fmt.Errorf("no destinations specified for EC shard distribution") - } - - destinations := t.destinations - - glog.V(1).Infof("Copying EC shards for volume %d to %d destinations", t.volumeID, len(destinations)) - - // Prepare shard IDs (0-13 for EC shards) - var shardIds []uint32 - for i := 0; i < erasure_coding.TotalShardsCount; i++ { - shardIds = append(shardIds, uint32(i)) - } - - // Distribute shards across destinations - var wg sync.WaitGroup - errorChan := make(chan error, len(destinations)) - - // Track which disks have already received metadata files (server+disk) - metadataFilesCopied := make(map[string]bool) - var metadataMutex sync.Mutex - - // For each destination, copy a subset of shards - shardsPerDest := len(shardIds) / len(destinations) - remainder := len(shardIds) % len(destinations) - - shardOffset := 0 - for i, dest := range destinations { - wg.Add(1) - - shardsForThisDest := shardsPerDest - if i < remainder { - shardsForThisDest++ // Distribute remainder shards - } - - destShardIds := shardIds[shardOffset : shardOffset+shardsForThisDest] - shardOffset += shardsForThisDest - - go func(destination *worker_pb.ECDestination, targetShardIds []uint32) { - defer wg.Done() - - if t.IsCancelled() { - errorChan <- fmt.Errorf("task cancelled during shard copy") - return - } - - // Create disk-specific metadata key (server+disk) - diskKey := fmt.Sprintf("%s:%d", destination.Node, destination.DiskId) - - glog.V(1).Infof("Copying shards %v from %s to %s (disk %d)", - targetShardIds, t.sourceServer, destination.Node, destination.DiskId) - - // Check if this disk needs metadata files (only once per disk) - metadataMutex.Lock() - needsMetadataFiles := !metadataFilesCopied[diskKey] - if needsMetadataFiles { - metadataFilesCopied[diskKey] = true - } - metadataMutex.Unlock() - - err := operation.WithVolumeServerClient(false, pb.ServerAddress(destination.Node), t.grpcDialOpt, - func(client volume_server_pb.VolumeServerClient) error { - _, copyErr := client.VolumeEcShardsCopy(context.Background(), &volume_server_pb.VolumeEcShardsCopyRequest{ - VolumeId: uint32(t.volumeID), - Collection: t.collection, - ShardIds: targetShardIds, - CopyEcxFile: needsMetadataFiles, // Copy .ecx only once per disk - CopyEcjFile: needsMetadataFiles, // Copy .ecj only once per disk - CopyVifFile: needsMetadataFiles, // Copy .vif only once per disk - SourceDataNode: t.sourceServer, - DiskId: destination.DiskId, // Pass target disk ID - }) - return copyErr - }) - - if err != nil { - errorChan <- fmt.Errorf("failed to copy shards to %s disk %d: %v", destination.Node, destination.DiskId, err) - return - } - - if needsMetadataFiles { - glog.V(1).Infof("Successfully copied shards %v and metadata files (.ecx, .ecj, .vif) to %s disk %d", - targetShardIds, destination.Node, destination.DiskId) - } else { - glog.V(1).Infof("Successfully copied shards %v to %s disk %d (metadata files already present)", - targetShardIds, destination.Node, destination.DiskId) - } - }(dest, destShardIds) - } - - wg.Wait() - close(errorChan) - - // Check for any copy errors - if err := <-errorChan; err != nil { - return err - } - - glog.V(1).Infof("Successfully copied all EC shards for volume %d", t.volumeID) - return nil -} - -// distributeEcShardsFromWorker distributes locally generated EC shards to destination servers -func (t *Task) distributeEcShardsFromWorker(localShardFiles map[string]string) error { - if len(t.destinations) == 0 { - return fmt.Errorf("no destinations specified for EC shard distribution") - } - - destinations := t.destinations - - glog.V(1).Infof("Distributing EC shards for volume %d from worker to %d destinations", t.volumeID, len(destinations)) - - // Prepare shard IDs (0-13 for EC shards) - var shardIds []uint32 - for i := 0; i < erasure_coding.TotalShardsCount; i++ { - shardIds = append(shardIds, uint32(i)) - } - - // Distribute shards across destinations - var wg sync.WaitGroup - errorChan := make(chan error, len(destinations)) - - // Track which disks have already received metadata files (server+disk) - metadataFilesCopied := make(map[string]bool) - var metadataMutex sync.Mutex - - // For each destination, send a subset of shards - shardsPerDest := len(shardIds) / len(destinations) - remainder := len(shardIds) % len(destinations) - - shardOffset := 0 - for i, dest := range destinations { - wg.Add(1) - - shardsForThisDest := shardsPerDest - if i < remainder { - shardsForThisDest++ // Distribute remainder shards - } - - destShardIds := shardIds[shardOffset : shardOffset+shardsForThisDest] - shardOffset += shardsForThisDest - - go func(destination *worker_pb.ECDestination, targetShardIds []uint32) { - defer wg.Done() - - if t.IsCancelled() { - errorChan <- fmt.Errorf("task cancelled during shard distribution") - return - } - - // Create disk-specific metadata key (server+disk) - diskKey := fmt.Sprintf("%s:%d", destination.Node, destination.DiskId) - - glog.V(1).Infof("Distributing shards %v from worker to %s (disk %d)", - targetShardIds, destination.Node, destination.DiskId) - - // Check if this disk needs metadata files (only once per disk) - metadataMutex.Lock() - needsMetadataFiles := !metadataFilesCopied[diskKey] - if needsMetadataFiles { - metadataFilesCopied[diskKey] = true - } - metadataMutex.Unlock() - - // Send shard files to destination using HTTP upload (simplified for now) - err := t.sendShardsToDestination(destination, targetShardIds, localShardFiles, needsMetadataFiles) - if err != nil { - errorChan <- fmt.Errorf("failed to send shards to %s disk %d: %v", destination.Node, destination.DiskId, err) - return - } - - if needsMetadataFiles { - glog.V(1).Infof("Successfully distributed shards %v and metadata files (.ecx, .vif) to %s disk %d", - targetShardIds, destination.Node, destination.DiskId) - } else { - glog.V(1).Infof("Successfully distributed shards %v to %s disk %d (metadata files already present)", - targetShardIds, destination.Node, destination.DiskId) - } - }(dest, destShardIds) - } - - wg.Wait() - close(errorChan) - - // Check for any distribution errors - if err := <-errorChan; err != nil { - return err - } - - glog.V(1).Infof("Completed distributing EC shards for volume %d", t.volumeID) - return nil -} - -// sendShardsToDestination sends specific shard files from worker to a destination server (simplified) -func (t *Task) sendShardsToDestination(destination *worker_pb.ECDestination, shardIds []uint32, localFiles map[string]string, includeMetadata bool) error { - // For now, use a simplified approach - just upload the files - // In a full implementation, this would use proper file upload mechanisms - glog.V(2).Infof("Would send shards %v and metadata=%v to %s disk %d", shardIds, includeMetadata, destination.Node, destination.DiskId) - - // TODO: Implement actual file upload to volume server - // This is a placeholder - actual implementation would: - // 1. Open each shard file locally - // 2. Upload via HTTP POST or gRPC stream to destination volume server - // 3. Volume server would save to the specified disk_id - - return nil -} - -// mountEcShardsOnDestinations mounts EC shards on all destination servers -func (t *Task) mountEcShardsOnDestinations() error { - if len(t.destinations) == 0 { - return fmt.Errorf("no destinations specified for mounting EC shards") - } - - destinations := t.destinations - - glog.V(1).Infof("Mounting EC shards for volume %d on %d destinations", t.volumeID, len(destinations)) - - // Prepare all shard IDs (0-13) - var allShardIds []uint32 - for i := 0; i < erasure_coding.TotalShardsCount; i++ { - allShardIds = append(allShardIds, uint32(i)) - } - - var wg sync.WaitGroup - errorChan := make(chan error, len(destinations)) - - // Mount shards on each destination server - for _, dest := range destinations { - wg.Add(1) - - go func(destination *worker_pb.ECDestination) { - defer wg.Done() - - if t.IsCancelled() { - errorChan <- fmt.Errorf("task cancelled during shard mounting") - return - } - - glog.V(1).Infof("Mounting EC shards on %s disk %d", destination.Node, destination.DiskId) - - err := operation.WithVolumeServerClient(false, pb.ServerAddress(destination.Node), t.grpcDialOpt, - func(client volume_server_pb.VolumeServerClient) error { - _, mountErr := client.VolumeEcShardsMount(context.Background(), &volume_server_pb.VolumeEcShardsMountRequest{ - VolumeId: uint32(t.volumeID), - Collection: t.collection, - ShardIds: allShardIds, // Mount all available shards on each server - }) - return mountErr - }) - - if err != nil { - // It's normal for some servers to not have all shards, so log as warning rather than error - glog.Warningf("Failed to mount some shards on %s disk %d (this may be normal): %v", destination.Node, destination.DiskId, err) - } else { - glog.V(1).Infof("Successfully mounted EC shards on %s disk %d", destination.Node, destination.DiskId) - } - }(dest) - } - - wg.Wait() - close(errorChan) - - // Check for any critical mounting errors - select { - case err := <-errorChan: - if err != nil { - glog.Warningf("Some shard mounting issues occurred: %v", err) - } - default: - // No errors - } - - glog.V(1).Infof("Completed mounting EC shards for volume %d", t.volumeID) - return nil -} - -// deleteVolumeFromAllLocations deletes the original volume from all replica servers -func (t *Task) deleteVolumeFromAllLocations(volumeId needle.VolumeId, volumeLocations []string) error { - glog.V(1).Infof("Deleting original volume %d from %d locations", volumeId, len(volumeLocations)) - - for _, location := range volumeLocations { - glog.V(1).Infof("Deleting volume %d from %s", volumeId, location) - - err := operation.WithVolumeServerClient(false, pb.ServerAddress(location), t.grpcDialOpt, - func(client volume_server_pb.VolumeServerClient) error { - _, deleteErr := client.VolumeDelete(context.Background(), &volume_server_pb.VolumeDeleteRequest{ - VolumeId: uint32(volumeId), - OnlyEmpty: false, // Force delete even if not empty since we've already created EC shards - }) - return deleteErr - }) - - if err != nil { - glog.Errorf("Failed to delete volume %d from %s: %v", volumeId, location, err) - return fmt.Errorf("failed to delete volume %d from %s: %v", volumeId, location, err) - } - - glog.V(1).Infof("Successfully deleted volume %d from %s", volumeId, location) - } - - glog.V(1).Infof("Successfully deleted volume %d from all %d locations", volumeId, len(volumeLocations)) - return nil -} - -// Register the task in the global registry -func init() { - types.RegisterGlobalTypedTask(types.TaskTypeErasureCoding, NewTask) - glog.V(1).Infof("Registered EC task") -} diff --git a/weed/worker/tasks/erasure_coding/ec_task.go b/weed/worker/tasks/erasure_coding/ec_task.go new file mode 100644 index 000000000..a6a3f749f --- /dev/null +++ b/weed/worker/tasks/erasure_coding/ec_task.go @@ -0,0 +1,660 @@ +package erasure_coding + +import ( + "context" + "fmt" + "io" + "math" + "os" + "path/filepath" + "sort" + "strings" + "time" + + "github.com/seaweedfs/seaweedfs/weed/glog" + "github.com/seaweedfs/seaweedfs/weed/operation" + "github.com/seaweedfs/seaweedfs/weed/pb" + "github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" + "github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding" + "github.com/seaweedfs/seaweedfs/weed/storage/needle" + "github.com/seaweedfs/seaweedfs/weed/storage/volume_info" + "github.com/seaweedfs/seaweedfs/weed/worker/types" + "github.com/seaweedfs/seaweedfs/weed/worker/types/base" + "google.golang.org/grpc" +) + +// ErasureCodingTask implements the Task interface +type ErasureCodingTask struct { + *base.BaseTask + server string + volumeID uint32 + collection string + workDir string + progress float64 + + // EC parameters + dataShards int32 + parityShards int32 + destinations []*worker_pb.ECDestination + shardAssignment map[string][]string // destination -> assigned shard types + replicas []string // volume replica servers for deletion +} + +// NewErasureCodingTask creates a new unified EC task instance +func NewErasureCodingTask(id string, server string, volumeID uint32, collection string) *ErasureCodingTask { + return &ErasureCodingTask{ + BaseTask: base.NewBaseTask(id, types.TaskTypeErasureCoding), + server: server, + volumeID: volumeID, + collection: collection, + dataShards: erasure_coding.DataShardsCount, // Default values + parityShards: erasure_coding.ParityShardsCount, // Default values + } +} + +// Execute implements the UnifiedTask interface +func (t *ErasureCodingTask) Execute(ctx context.Context, params *worker_pb.TaskParams) error { + if params == nil { + return fmt.Errorf("task parameters are required") + } + + ecParams := params.GetErasureCodingParams() + if ecParams == nil { + return fmt.Errorf("erasure coding parameters are required") + } + + t.dataShards = ecParams.DataShards + t.parityShards = ecParams.ParityShards + t.workDir = ecParams.WorkingDir + t.destinations = ecParams.Destinations + t.replicas = params.Replicas // Get replicas from task parameters + + t.GetLogger().WithFields(map[string]interface{}{ + "volume_id": t.volumeID, + "server": t.server, + "collection": t.collection, + "data_shards": t.dataShards, + "parity_shards": t.parityShards, + "destinations": len(t.destinations), + }).Info("Starting erasure coding task") + + // Use the working directory from task parameters, or fall back to a default + baseWorkDir := t.workDir + if baseWorkDir == "" { + baseWorkDir = "/tmp/seaweedfs_ec_work" + } + + // Create unique working directory for this task + taskWorkDir := filepath.Join(baseWorkDir, fmt.Sprintf("vol_%d_%d", t.volumeID, time.Now().Unix())) + if err := os.MkdirAll(taskWorkDir, 0755); err != nil { + return fmt.Errorf("failed to create task working directory %s: %v", taskWorkDir, err) + } + glog.V(1).Infof("Created working directory: %s", taskWorkDir) + + // Update the task's working directory to the specific instance directory + t.workDir = taskWorkDir + glog.V(1).Infof("Task working directory configured: %s (logs will be written here)", taskWorkDir) + + // Ensure cleanup of working directory (but preserve logs) + defer func() { + // Clean up volume files and EC shards, but preserve the directory structure and any logs + patterns := []string{"*.dat", "*.idx", "*.ec*", "*.vif"} + for _, pattern := range patterns { + matches, err := filepath.Glob(filepath.Join(taskWorkDir, pattern)) + if err != nil { + continue + } + for _, match := range matches { + if err := os.Remove(match); err != nil { + glog.V(2).Infof("Could not remove %s: %v", match, err) + } + } + } + glog.V(1).Infof("Cleaned up volume files from working directory: %s (logs preserved)", taskWorkDir) + }() + + // Step 1: Mark volume readonly + t.ReportProgress(10.0) + t.GetLogger().Info("Marking volume readonly") + if err := t.markVolumeReadonly(); err != nil { + return fmt.Errorf("failed to mark volume readonly: %v", err) + } + + // Step 2: Copy volume files to worker + t.ReportProgress(25.0) + t.GetLogger().Info("Copying volume files to worker") + localFiles, err := t.copyVolumeFilesToWorker(taskWorkDir) + if err != nil { + return fmt.Errorf("failed to copy volume files: %v", err) + } + + // Step 3: Generate EC shards locally + t.ReportProgress(40.0) + t.GetLogger().Info("Generating EC shards locally") + shardFiles, err := t.generateEcShardsLocally(localFiles, taskWorkDir) + if err != nil { + return fmt.Errorf("failed to generate EC shards: %v", err) + } + + // Step 4: Distribute shards to destinations + t.ReportProgress(60.0) + t.GetLogger().Info("Distributing EC shards to destinations") + if err := t.distributeEcShards(shardFiles); err != nil { + return fmt.Errorf("failed to distribute EC shards: %v", err) + } + + // Step 5: Mount EC shards + t.ReportProgress(80.0) + t.GetLogger().Info("Mounting EC shards") + if err := t.mountEcShards(); err != nil { + return fmt.Errorf("failed to mount EC shards: %v", err) + } + + // Step 6: Delete original volume + t.ReportProgress(90.0) + t.GetLogger().Info("Deleting original volume") + if err := t.deleteOriginalVolume(); err != nil { + return fmt.Errorf("failed to delete original volume: %v", err) + } + + t.ReportProgress(100.0) + glog.Infof("EC task completed successfully: volume %d from %s with %d shards distributed", + t.volumeID, t.server, len(shardFiles)) + + return nil +} + +// Validate implements the UnifiedTask interface +func (t *ErasureCodingTask) Validate(params *worker_pb.TaskParams) error { + if params == nil { + return fmt.Errorf("task parameters are required") + } + + ecParams := params.GetErasureCodingParams() + if ecParams == nil { + return fmt.Errorf("erasure coding parameters are required") + } + + if params.VolumeId != t.volumeID { + return fmt.Errorf("volume ID mismatch: expected %d, got %d", t.volumeID, params.VolumeId) + } + + if params.Server != t.server { + return fmt.Errorf("source server mismatch: expected %s, got %s", t.server, params.Server) + } + + if ecParams.DataShards < 1 { + return fmt.Errorf("invalid data shards: %d (must be >= 1)", ecParams.DataShards) + } + + if ecParams.ParityShards < 1 { + return fmt.Errorf("invalid parity shards: %d (must be >= 1)", ecParams.ParityShards) + } + + if len(ecParams.Destinations) < int(ecParams.DataShards+ecParams.ParityShards) { + return fmt.Errorf("insufficient destinations: got %d, need %d", len(ecParams.Destinations), ecParams.DataShards+ecParams.ParityShards) + } + + return nil +} + +// EstimateTime implements the UnifiedTask interface +func (t *ErasureCodingTask) EstimateTime(params *worker_pb.TaskParams) time.Duration { + // Basic estimate based on simulated steps + return 20 * time.Second // Sum of all step durations +} + +// GetProgress returns current progress +func (t *ErasureCodingTask) GetProgress() float64 { + return t.progress +} + +// Helper methods for actual EC operations + +// markVolumeReadonly marks the volume as readonly on the source server +func (t *ErasureCodingTask) markVolumeReadonly() error { + return operation.WithVolumeServerClient(false, pb.ServerAddress(t.server), grpc.WithInsecure(), + func(client volume_server_pb.VolumeServerClient) error { + _, err := client.VolumeMarkReadonly(context.Background(), &volume_server_pb.VolumeMarkReadonlyRequest{ + VolumeId: t.volumeID, + }) + return err + }) +} + +// copyVolumeFilesToWorker copies .dat and .idx files from source server to local worker +func (t *ErasureCodingTask) copyVolumeFilesToWorker(workDir string) (map[string]string, error) { + localFiles := make(map[string]string) + + // Copy .dat file + datFile := filepath.Join(workDir, fmt.Sprintf("%d.dat", t.volumeID)) + if err := t.copyFileFromSource(".dat", datFile); err != nil { + return nil, fmt.Errorf("failed to copy .dat file: %v", err) + } + localFiles["dat"] = datFile + + // Copy .idx file + idxFile := filepath.Join(workDir, fmt.Sprintf("%d.idx", t.volumeID)) + if err := t.copyFileFromSource(".idx", idxFile); err != nil { + return nil, fmt.Errorf("failed to copy .idx file: %v", err) + } + localFiles["idx"] = idxFile + + return localFiles, nil +} + +// copyFileFromSource copies a file from source server to local path using gRPC streaming +func (t *ErasureCodingTask) copyFileFromSource(ext, localPath string) error { + return operation.WithVolumeServerClient(false, pb.ServerAddress(t.server), grpc.WithInsecure(), + func(client volume_server_pb.VolumeServerClient) error { + stream, err := client.CopyFile(context.Background(), &volume_server_pb.CopyFileRequest{ + VolumeId: t.volumeID, + Collection: t.collection, + Ext: ext, + StopOffset: uint64(math.MaxInt64), + }) + if err != nil { + return fmt.Errorf("failed to initiate file copy: %v", err) + } + + // Create local file + localFile, err := os.Create(localPath) + if err != nil { + return fmt.Errorf("failed to create local file %s: %v", localPath, err) + } + defer localFile.Close() + + // Stream data and write to local file + totalBytes := int64(0) + for { + resp, err := stream.Recv() + if err == io.EOF { + break + } + if err != nil { + return fmt.Errorf("failed to receive file data: %v", err) + } + + if len(resp.FileContent) > 0 { + written, writeErr := localFile.Write(resp.FileContent) + if writeErr != nil { + return fmt.Errorf("failed to write to local file: %v", writeErr) + } + totalBytes += int64(written) + } + } + + glog.V(1).Infof("Successfully copied %s (%d bytes) from %s to %s", ext, totalBytes, t.server, localPath) + return nil + }) +} + +// generateEcShardsLocally generates EC shards from local volume files +func (t *ErasureCodingTask) generateEcShardsLocally(localFiles map[string]string, workDir string) (map[string]string, error) { + datFile := localFiles["dat"] + idxFile := localFiles["idx"] + + if datFile == "" || idxFile == "" { + return nil, fmt.Errorf("missing required volume files: dat=%s, idx=%s", datFile, idxFile) + } + + // Get base name without extension for EC operations + baseName := strings.TrimSuffix(datFile, ".dat") + shardFiles := make(map[string]string) + + glog.V(1).Infof("Generating EC shards from local files: dat=%s, idx=%s", datFile, idxFile) + + // Generate EC shard files (.ec00 ~ .ec13) + if err := erasure_coding.WriteEcFiles(baseName); err != nil { + return nil, fmt.Errorf("failed to generate EC shard files: %v", err) + } + + // Generate .ecx file from .idx (use baseName, not full idx path) + if err := erasure_coding.WriteSortedFileFromIdx(baseName, ".ecx"); err != nil { + return nil, fmt.Errorf("failed to generate .ecx file: %v", err) + } + + // Collect generated shard file paths + for i := 0; i < erasure_coding.TotalShardsCount; i++ { + shardFile := fmt.Sprintf("%s.ec%02d", baseName, i) + if _, err := os.Stat(shardFile); err == nil { + shardFiles[fmt.Sprintf("ec%02d", i)] = shardFile + } + } + + // Add metadata files + ecxFile := baseName + ".ecx" + if _, err := os.Stat(ecxFile); err == nil { + shardFiles["ecx"] = ecxFile + } + + // Generate .vif file (volume info) + vifFile := baseName + ".vif" + volumeInfo := &volume_server_pb.VolumeInfo{ + Version: uint32(needle.GetCurrentVersion()), + } + if err := volume_info.SaveVolumeInfo(vifFile, volumeInfo); err != nil { + glog.Warningf("Failed to create .vif file: %v", err) + } else { + shardFiles["vif"] = vifFile + } + + glog.V(1).Infof("Generated %d EC files locally", len(shardFiles)) + return shardFiles, nil +} + +// distributeEcShards distributes locally generated EC shards to destination servers +func (t *ErasureCodingTask) distributeEcShards(shardFiles map[string]string) error { + if len(t.destinations) == 0 { + return fmt.Errorf("no destinations specified for EC shard distribution") + } + + if len(shardFiles) == 0 { + return fmt.Errorf("no shard files available for distribution") + } + + // Create shard assignment: assign specific shards to specific destinations + shardAssignment := t.createShardAssignment(shardFiles) + if len(shardAssignment) == 0 { + return fmt.Errorf("failed to create shard assignment") + } + + // Store assignment for use during mounting + t.shardAssignment = shardAssignment + + // Send assigned shards to each destination + for destNode, assignedShards := range shardAssignment { + t.GetLogger().WithFields(map[string]interface{}{ + "destination": destNode, + "assigned_shards": len(assignedShards), + "shard_ids": assignedShards, + }).Info("Distributing assigned EC shards to destination") + + // Send only the assigned shards to this destination + for _, shardType := range assignedShards { + filePath, exists := shardFiles[shardType] + if !exists { + return fmt.Errorf("shard file %s not found for destination %s", shardType, destNode) + } + + if err := t.sendShardFileToDestination(destNode, filePath, shardType); err != nil { + return fmt.Errorf("failed to send %s to %s: %v", shardType, destNode, err) + } + } + } + + glog.V(1).Infof("Successfully distributed EC shards to %d destinations", len(shardAssignment)) + return nil +} + +// createShardAssignment assigns specific EC shards to specific destination servers +// Each destination gets a subset of shards based on availability and placement rules +func (t *ErasureCodingTask) createShardAssignment(shardFiles map[string]string) map[string][]string { + assignment := make(map[string][]string) + + // Collect all available EC shards (ec00-ec13) + var availableShards []string + for shardType := range shardFiles { + if strings.HasPrefix(shardType, "ec") && len(shardType) == 4 { + availableShards = append(availableShards, shardType) + } + } + + // Sort shards for consistent assignment + sort.Strings(availableShards) + + if len(availableShards) == 0 { + glog.Warningf("No EC shards found for assignment") + return assignment + } + + // Calculate shards per destination + numDestinations := len(t.destinations) + if numDestinations == 0 { + return assignment + } + + // Strategy: Distribute shards as evenly as possible across destinations + // With 14 shards and N destinations, some destinations get ⌈14/N⌉ shards, others get ⌊14/N⌋ + shardsPerDest := len(availableShards) / numDestinations + extraShards := len(availableShards) % numDestinations + + shardIndex := 0 + for i, dest := range t.destinations { + var destShards []string + + // Assign base number of shards + shardsToAssign := shardsPerDest + + // Assign one extra shard to first 'extraShards' destinations + if i < extraShards { + shardsToAssign++ + } + + // Assign the shards + for j := 0; j < shardsToAssign && shardIndex < len(availableShards); j++ { + destShards = append(destShards, availableShards[shardIndex]) + shardIndex++ + } + + assignment[dest.Node] = destShards + + glog.V(2).Infof("Assigned shards %v to destination %s", destShards, dest.Node) + } + + // Assign metadata files (.ecx, .vif) to each destination that has shards + // Note: .ecj files are created during mount, not during initial generation + for destNode, destShards := range assignment { + if len(destShards) > 0 { + // Add .ecx file if available + if _, hasEcx := shardFiles["ecx"]; hasEcx { + assignment[destNode] = append(assignment[destNode], "ecx") + } + + // Add .vif file if available + if _, hasVif := shardFiles["vif"]; hasVif { + assignment[destNode] = append(assignment[destNode], "vif") + } + + glog.V(2).Infof("Assigned metadata files (.ecx, .vif) to destination %s", destNode) + } + } + + return assignment +} + +// sendShardFileToDestination sends a single shard file to a destination server using ReceiveFile API +func (t *ErasureCodingTask) sendShardFileToDestination(destServer, filePath, shardType string) error { + return operation.WithVolumeServerClient(false, pb.ServerAddress(destServer), grpc.WithInsecure(), + func(client volume_server_pb.VolumeServerClient) error { + // Open the local shard file + file, err := os.Open(filePath) + if err != nil { + return fmt.Errorf("failed to open shard file %s: %v", filePath, err) + } + defer file.Close() + + // Get file size + fileInfo, err := file.Stat() + if err != nil { + return fmt.Errorf("failed to get file info for %s: %v", filePath, err) + } + + // Determine file extension and shard ID + var ext string + var shardId uint32 + if shardType == "ecx" { + ext = ".ecx" + shardId = 0 // ecx file doesn't have a specific shard ID + } else if shardType == "vif" { + ext = ".vif" + shardId = 0 // vif file doesn't have a specific shard ID + } else if strings.HasPrefix(shardType, "ec") && len(shardType) == 4 { + // EC shard file like "ec00", "ec01", etc. + ext = "." + shardType + fmt.Sscanf(shardType[2:], "%d", &shardId) + } else { + return fmt.Errorf("unknown shard type: %s", shardType) + } + + // Create streaming client + stream, err := client.ReceiveFile(context.Background()) + if err != nil { + return fmt.Errorf("failed to create receive stream: %v", err) + } + + // Send file info first + err = stream.Send(&volume_server_pb.ReceiveFileRequest{ + Data: &volume_server_pb.ReceiveFileRequest_Info{ + Info: &volume_server_pb.ReceiveFileInfo{ + VolumeId: t.volumeID, + Ext: ext, + Collection: t.collection, + IsEcVolume: true, + ShardId: shardId, + FileSize: uint64(fileInfo.Size()), + }, + }, + }) + if err != nil { + return fmt.Errorf("failed to send file info: %v", err) + } + + // Send file content in chunks + buffer := make([]byte, 64*1024) // 64KB chunks + for { + n, readErr := file.Read(buffer) + if n > 0 { + err = stream.Send(&volume_server_pb.ReceiveFileRequest{ + Data: &volume_server_pb.ReceiveFileRequest_FileContent{ + FileContent: buffer[:n], + }, + }) + if err != nil { + return fmt.Errorf("failed to send file content: %v", err) + } + } + if readErr == io.EOF { + break + } + if readErr != nil { + return fmt.Errorf("failed to read file: %v", readErr) + } + } + + // Close stream and get response + resp, err := stream.CloseAndRecv() + if err != nil { + return fmt.Errorf("failed to close stream: %v", err) + } + + if resp.Error != "" { + return fmt.Errorf("server error: %s", resp.Error) + } + + glog.V(2).Infof("Successfully sent %s (%d bytes) to %s", shardType, resp.BytesWritten, destServer) + return nil + }) +} + +// mountEcShards mounts EC shards on destination servers +func (t *ErasureCodingTask) mountEcShards() error { + if t.shardAssignment == nil { + return fmt.Errorf("shard assignment not available for mounting") + } + + // Mount only assigned shards on each destination + for destNode, assignedShards := range t.shardAssignment { + // Convert shard names to shard IDs for mounting + var shardIds []uint32 + for _, shardType := range assignedShards { + // Skip metadata files (.ecx, .vif) - only mount EC shards + if strings.HasPrefix(shardType, "ec") && len(shardType) == 4 { + // Parse shard ID from "ec00", "ec01", etc. + var shardId uint32 + if _, err := fmt.Sscanf(shardType[2:], "%d", &shardId); err == nil { + shardIds = append(shardIds, shardId) + } + } + } + + if len(shardIds) == 0 { + glog.V(1).Infof("No EC shards to mount on %s (only metadata files)", destNode) + continue + } + + glog.V(1).Infof("Mounting shards %v on %s", shardIds, destNode) + + err := operation.WithVolumeServerClient(false, pb.ServerAddress(destNode), grpc.WithInsecure(), + func(client volume_server_pb.VolumeServerClient) error { + _, mountErr := client.VolumeEcShardsMount(context.Background(), &volume_server_pb.VolumeEcShardsMountRequest{ + VolumeId: t.volumeID, + Collection: t.collection, + ShardIds: shardIds, + }) + return mountErr + }) + + if err != nil { + glog.Warningf("Failed to mount shards %v on %s: %v", shardIds, destNode, err) + } else { + glog.V(1).Infof("Successfully mounted EC shards %v on %s", shardIds, destNode) + } + } + + return nil +} + +// deleteOriginalVolume deletes the original volume and all its replicas from all servers +func (t *ErasureCodingTask) deleteOriginalVolume() error { + // Get replicas from task parameters (set during detection) + replicas := t.getReplicas() + + if len(replicas) == 0 { + glog.Warningf("No replicas found for volume %d, falling back to source server only", t.volumeID) + replicas = []string{t.server} + } + + glog.V(1).Infof("Deleting volume %d from %d replica servers: %v", t.volumeID, len(replicas), replicas) + + // Delete volume from all replica locations + var deleteErrors []string + successCount := 0 + + for _, replicaServer := range replicas { + err := operation.WithVolumeServerClient(false, pb.ServerAddress(replicaServer), grpc.WithInsecure(), + func(client volume_server_pb.VolumeServerClient) error { + _, err := client.VolumeDelete(context.Background(), &volume_server_pb.VolumeDeleteRequest{ + VolumeId: t.volumeID, + OnlyEmpty: false, // Force delete since we've created EC shards + }) + return err + }) + + if err != nil { + deleteErrors = append(deleteErrors, fmt.Sprintf("failed to delete volume %d from %s: %v", t.volumeID, replicaServer, err)) + glog.Warningf("Failed to delete volume %d from replica server %s: %v", t.volumeID, replicaServer, err) + } else { + successCount++ + glog.V(1).Infof("Successfully deleted volume %d from replica server %s", t.volumeID, replicaServer) + } + } + + // Report results + if len(deleteErrors) > 0 { + glog.Warningf("Some volume deletions failed (%d/%d successful): %v", successCount, len(replicas), deleteErrors) + // Don't return error - EC task should still be considered successful if shards are mounted + } else { + glog.V(1).Infof("Successfully deleted volume %d from all %d replica servers", t.volumeID, len(replicas)) + } + + return nil +} + +// getReplicas extracts replica servers from task parameters +func (t *ErasureCodingTask) getReplicas() []string { + // Access replicas from the parameters passed during Execute + // We'll need to store these during Execute - let me add a field to the task + return t.replicas +} diff --git a/weed/worker/tasks/erasure_coding/monitoring.go b/weed/worker/tasks/erasure_coding/monitoring.go new file mode 100644 index 000000000..799eb62c8 --- /dev/null +++ b/weed/worker/tasks/erasure_coding/monitoring.go @@ -0,0 +1,229 @@ +package erasure_coding + +import ( + "sync" + "time" +) + +// ErasureCodingMetrics contains erasure coding-specific monitoring data +type ErasureCodingMetrics struct { + // Execution metrics + VolumesEncoded int64 `json:"volumes_encoded"` + TotalShardsCreated int64 `json:"total_shards_created"` + TotalDataProcessed int64 `json:"total_data_processed"` + TotalSourcesRemoved int64 `json:"total_sources_removed"` + LastEncodingTime time.Time `json:"last_encoding_time"` + + // Performance metrics + AverageEncodingTime int64 `json:"average_encoding_time_seconds"` + AverageShardSize int64 `json:"average_shard_size"` + AverageDataShards int `json:"average_data_shards"` + AverageParityShards int `json:"average_parity_shards"` + SuccessfulOperations int64 `json:"successful_operations"` + FailedOperations int64 `json:"failed_operations"` + + // Distribution metrics + ShardsPerDataCenter map[string]int64 `json:"shards_per_datacenter"` + ShardsPerRack map[string]int64 `json:"shards_per_rack"` + PlacementSuccessRate float64 `json:"placement_success_rate"` + + // Current task metrics + CurrentVolumeSize int64 `json:"current_volume_size"` + CurrentShardCount int `json:"current_shard_count"` + VolumesPendingEncoding int `json:"volumes_pending_encoding"` + + mutex sync.RWMutex +} + +// NewErasureCodingMetrics creates a new erasure coding metrics instance +func NewErasureCodingMetrics() *ErasureCodingMetrics { + return &ErasureCodingMetrics{ + LastEncodingTime: time.Now(), + ShardsPerDataCenter: make(map[string]int64), + ShardsPerRack: make(map[string]int64), + } +} + +// RecordVolumeEncoded records a successful volume encoding operation +func (m *ErasureCodingMetrics) RecordVolumeEncoded(volumeSize int64, shardsCreated int, dataShards int, parityShards int, encodingTime time.Duration, sourceRemoved bool) { + m.mutex.Lock() + defer m.mutex.Unlock() + + m.VolumesEncoded++ + m.TotalShardsCreated += int64(shardsCreated) + m.TotalDataProcessed += volumeSize + m.SuccessfulOperations++ + m.LastEncodingTime = time.Now() + + if sourceRemoved { + m.TotalSourcesRemoved++ + } + + // Update average encoding time + if m.AverageEncodingTime == 0 { + m.AverageEncodingTime = int64(encodingTime.Seconds()) + } else { + // Exponential moving average + newTime := int64(encodingTime.Seconds()) + m.AverageEncodingTime = (m.AverageEncodingTime*4 + newTime) / 5 + } + + // Update average shard size + if shardsCreated > 0 { + avgShardSize := volumeSize / int64(shardsCreated) + if m.AverageShardSize == 0 { + m.AverageShardSize = avgShardSize + } else { + m.AverageShardSize = (m.AverageShardSize*4 + avgShardSize) / 5 + } + } + + // Update average data/parity shards + if m.AverageDataShards == 0 { + m.AverageDataShards = dataShards + m.AverageParityShards = parityShards + } else { + m.AverageDataShards = (m.AverageDataShards*4 + dataShards) / 5 + m.AverageParityShards = (m.AverageParityShards*4 + parityShards) / 5 + } +} + +// RecordFailure records a failed erasure coding operation +func (m *ErasureCodingMetrics) RecordFailure() { + m.mutex.Lock() + defer m.mutex.Unlock() + + m.FailedOperations++ +} + +// RecordShardPlacement records shard placement for distribution tracking +func (m *ErasureCodingMetrics) RecordShardPlacement(dataCenter string, rack string) { + m.mutex.Lock() + defer m.mutex.Unlock() + + m.ShardsPerDataCenter[dataCenter]++ + rackKey := dataCenter + ":" + rack + m.ShardsPerRack[rackKey]++ +} + +// UpdateCurrentVolumeInfo updates current volume processing information +func (m *ErasureCodingMetrics) UpdateCurrentVolumeInfo(volumeSize int64, shardCount int) { + m.mutex.Lock() + defer m.mutex.Unlock() + + m.CurrentVolumeSize = volumeSize + m.CurrentShardCount = shardCount +} + +// SetVolumesPendingEncoding sets the number of volumes pending encoding +func (m *ErasureCodingMetrics) SetVolumesPendingEncoding(count int) { + m.mutex.Lock() + defer m.mutex.Unlock() + + m.VolumesPendingEncoding = count +} + +// UpdatePlacementSuccessRate updates the placement success rate +func (m *ErasureCodingMetrics) UpdatePlacementSuccessRate(rate float64) { + m.mutex.Lock() + defer m.mutex.Unlock() + + if m.PlacementSuccessRate == 0 { + m.PlacementSuccessRate = rate + } else { + // Exponential moving average + m.PlacementSuccessRate = 0.8*m.PlacementSuccessRate + 0.2*rate + } +} + +// GetMetrics returns a copy of the current metrics (without the mutex) +func (m *ErasureCodingMetrics) GetMetrics() ErasureCodingMetrics { + m.mutex.RLock() + defer m.mutex.RUnlock() + + // Create deep copy of maps + shardsPerDC := make(map[string]int64) + for k, v := range m.ShardsPerDataCenter { + shardsPerDC[k] = v + } + + shardsPerRack := make(map[string]int64) + for k, v := range m.ShardsPerRack { + shardsPerRack[k] = v + } + + // Create a copy without the mutex to avoid copying lock value + return ErasureCodingMetrics{ + VolumesEncoded: m.VolumesEncoded, + TotalShardsCreated: m.TotalShardsCreated, + TotalDataProcessed: m.TotalDataProcessed, + TotalSourcesRemoved: m.TotalSourcesRemoved, + LastEncodingTime: m.LastEncodingTime, + AverageEncodingTime: m.AverageEncodingTime, + AverageShardSize: m.AverageShardSize, + AverageDataShards: m.AverageDataShards, + AverageParityShards: m.AverageParityShards, + SuccessfulOperations: m.SuccessfulOperations, + FailedOperations: m.FailedOperations, + ShardsPerDataCenter: shardsPerDC, + ShardsPerRack: shardsPerRack, + PlacementSuccessRate: m.PlacementSuccessRate, + CurrentVolumeSize: m.CurrentVolumeSize, + CurrentShardCount: m.CurrentShardCount, + VolumesPendingEncoding: m.VolumesPendingEncoding, + } +} + +// GetSuccessRate returns the success rate as a percentage +func (m *ErasureCodingMetrics) GetSuccessRate() float64 { + m.mutex.RLock() + defer m.mutex.RUnlock() + + total := m.SuccessfulOperations + m.FailedOperations + if total == 0 { + return 100.0 + } + return float64(m.SuccessfulOperations) / float64(total) * 100.0 +} + +// GetAverageDataProcessed returns the average data processed per volume +func (m *ErasureCodingMetrics) GetAverageDataProcessed() float64 { + m.mutex.RLock() + defer m.mutex.RUnlock() + + if m.VolumesEncoded == 0 { + return 0 + } + return float64(m.TotalDataProcessed) / float64(m.VolumesEncoded) +} + +// GetSourceRemovalRate returns the percentage of sources removed after encoding +func (m *ErasureCodingMetrics) GetSourceRemovalRate() float64 { + m.mutex.RLock() + defer m.mutex.RUnlock() + + if m.VolumesEncoded == 0 { + return 0 + } + return float64(m.TotalSourcesRemoved) / float64(m.VolumesEncoded) * 100.0 +} + +// Reset resets all metrics to zero +func (m *ErasureCodingMetrics) Reset() { + m.mutex.Lock() + defer m.mutex.Unlock() + + *m = ErasureCodingMetrics{ + LastEncodingTime: time.Now(), + ShardsPerDataCenter: make(map[string]int64), + ShardsPerRack: make(map[string]int64), + } +} + +// Global metrics instance for erasure coding tasks +var globalErasureCodingMetrics = NewErasureCodingMetrics() + +// GetGlobalErasureCodingMetrics returns the global erasure coding metrics instance +func GetGlobalErasureCodingMetrics() *ErasureCodingMetrics { + return globalErasureCodingMetrics +} diff --git a/weed/worker/tasks/erasure_coding/ec_register.go b/weed/worker/tasks/erasure_coding/register.go similarity index 83% rename from weed/worker/tasks/erasure_coding/ec_register.go rename to weed/worker/tasks/erasure_coding/register.go index 62cfe6b56..883aaf965 100644 --- a/weed/worker/tasks/erasure_coding/ec_register.go +++ b/weed/worker/tasks/erasure_coding/register.go @@ -5,6 +5,7 @@ import ( "time" "github.com/seaweedfs/seaweedfs/weed/glog" + "github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" "github.com/seaweedfs/seaweedfs/weed/worker/tasks" "github.com/seaweedfs/seaweedfs/weed/worker/tasks/base" "github.com/seaweedfs/seaweedfs/weed/worker/types" @@ -35,9 +36,19 @@ func RegisterErasureCodingTask() { Icon: "fas fa-shield-alt text-success", Capabilities: []string{"erasure_coding", "data_protection"}, - Config: config, - ConfigSpec: GetConfigSpec(), - CreateTask: nil, // Uses typed task system - see init() in ec.go + Config: config, + ConfigSpec: GetConfigSpec(), + CreateTask: func(params *worker_pb.TaskParams) (types.Task, error) { + if params == nil { + return nil, fmt.Errorf("task parameters are required") + } + return NewErasureCodingTask( + fmt.Sprintf("erasure_coding-%d", params.VolumeId), + params.Server, + params.VolumeId, + params.Collection, + ), nil + }, DetectionFunc: Detection, ScanInterval: 1 * time.Hour, SchedulingFunc: Scheduling, diff --git a/weed/worker/tasks/erasure_coding/scheduling.go b/weed/worker/tasks/erasure_coding/scheduling.go new file mode 100644 index 000000000..d9d891e04 --- /dev/null +++ b/weed/worker/tasks/erasure_coding/scheduling.go @@ -0,0 +1,40 @@ +package erasure_coding + +import ( + "github.com/seaweedfs/seaweedfs/weed/worker/tasks/base" + "github.com/seaweedfs/seaweedfs/weed/worker/types" +) + +// Scheduling implements the scheduling logic for erasure coding tasks +func Scheduling(task *types.TaskInput, runningTasks []*types.TaskInput, availableWorkers []*types.WorkerData, config base.TaskConfig) bool { + ecConfig := config.(*Config) + + // Check if we have available workers + if len(availableWorkers) == 0 { + return false + } + + // Count running EC tasks + runningCount := 0 + for _, runningTask := range runningTasks { + if runningTask.Type == types.TaskTypeErasureCoding { + runningCount++ + } + } + + // Check concurrency limit + if runningCount >= ecConfig.MaxConcurrent { + return false + } + + // Check if any worker can handle EC tasks + for _, worker := range availableWorkers { + for _, capability := range worker.Capabilities { + if capability == types.TaskTypeErasureCoding { + return true + } + } + } + + return false +} diff --git a/weed/worker/tasks/registry.go b/weed/worker/tasks/registry.go index 105055128..626a54a14 100644 --- a/weed/worker/tasks/registry.go +++ b/weed/worker/tasks/registry.go @@ -8,23 +8,14 @@ import ( ) var ( - globalRegistry *TaskRegistry globalTypesRegistry *types.TaskRegistry globalUIRegistry *types.UIRegistry - registryOnce sync.Once + globalTaskRegistry *TaskRegistry typesRegistryOnce sync.Once uiRegistryOnce sync.Once + taskRegistryOnce sync.Once ) -// GetGlobalRegistry returns the global task registry (singleton) -func GetGlobalRegistry() *TaskRegistry { - registryOnce.Do(func() { - globalRegistry = NewTaskRegistry() - glog.V(1).Infof("Created global task registry") - }) - return globalRegistry -} - // GetGlobalTypesRegistry returns the global types registry (singleton) func GetGlobalTypesRegistry() *types.TaskRegistry { typesRegistryOnce.Do(func() { @@ -43,9 +34,18 @@ func GetGlobalUIRegistry() *types.UIRegistry { return globalUIRegistry } -// AutoRegister registers a task directly with the global registry +// GetGlobalTaskRegistry returns the global task registry (singleton) +func GetGlobalTaskRegistry() *TaskRegistry { + taskRegistryOnce.Do(func() { + globalTaskRegistry = NewTaskRegistry() + glog.V(1).Infof("Created global task registry") + }) + return globalTaskRegistry +} + +// AutoRegister registers a task with the global task registry func AutoRegister(taskType types.TaskType, factory types.TaskFactory) { - registry := GetGlobalRegistry() + registry := GetGlobalTaskRegistry() registry.Register(taskType, factory) glog.V(1).Infof("Auto-registered task type: %s", taskType) } @@ -108,3 +108,41 @@ func SetMaintenancePolicyFromTasks() { // For now, we'll just log that this should be called by the integration layer glog.V(1).Infof("SetMaintenancePolicyFromTasks called - policy should be built by the integration layer") } + +// TaskRegistry manages task factories +type TaskRegistry struct { + factories map[types.TaskType]types.TaskFactory + mutex sync.RWMutex +} + +// NewTaskRegistry creates a new task registry +func NewTaskRegistry() *TaskRegistry { + return &TaskRegistry{ + factories: make(map[types.TaskType]types.TaskFactory), + } +} + +// Register adds a factory to the registry +func (r *TaskRegistry) Register(taskType types.TaskType, factory types.TaskFactory) { + r.mutex.Lock() + defer r.mutex.Unlock() + r.factories[taskType] = factory +} + +// Get returns a factory from the registry +func (r *TaskRegistry) Get(taskType types.TaskType) types.TaskFactory { + r.mutex.RLock() + defer r.mutex.RUnlock() + return r.factories[taskType] +} + +// GetAll returns all registered factories +func (r *TaskRegistry) GetAll() map[types.TaskType]types.TaskFactory { + r.mutex.RLock() + defer r.mutex.RUnlock() + result := make(map[types.TaskType]types.TaskFactory) + for k, v := range r.factories { + result[k] = v + } + return result +} diff --git a/weed/worker/tasks/task.go b/weed/worker/tasks/task.go index 15369c137..9813ae97f 100644 --- a/weed/worker/tasks/task.go +++ b/weed/worker/tasks/task.go @@ -318,59 +318,6 @@ func (t *BaseTask) ExecuteTask(ctx context.Context, params types.TaskParams, exe return nil } -// TaskRegistry manages task factories -type TaskRegistry struct { - factories map[types.TaskType]types.TaskFactory - mutex sync.RWMutex -} - -// NewTaskRegistry creates a new task registry -func NewTaskRegistry() *TaskRegistry { - return &TaskRegistry{ - factories: make(map[types.TaskType]types.TaskFactory), - } -} - -// Register registers a task factory -func (r *TaskRegistry) Register(taskType types.TaskType, factory types.TaskFactory) { - r.mutex.Lock() - defer r.mutex.Unlock() - r.factories[taskType] = factory -} - -// CreateTask creates a task instance -func (r *TaskRegistry) CreateTask(taskType types.TaskType, params types.TaskParams) (types.TaskInterface, error) { - r.mutex.RLock() - factory, exists := r.factories[taskType] - r.mutex.RUnlock() - - if !exists { - return nil, &UnsupportedTaskTypeError{TaskType: taskType} - } - - return factory.Create(params) -} - -// GetSupportedTypes returns all supported task types -func (r *TaskRegistry) GetSupportedTypes() []types.TaskType { - r.mutex.RLock() - defer r.mutex.RUnlock() - - types := make([]types.TaskType, 0, len(r.factories)) - for taskType := range r.factories { - types = append(types, taskType) - } - return types -} - -// GetFactory returns the factory for a task type -func (r *TaskRegistry) GetFactory(taskType types.TaskType) (types.TaskFactory, bool) { - r.mutex.RLock() - defer r.mutex.RUnlock() - factory, exists := r.factories[taskType] - return factory, exists -} - // UnsupportedTaskTypeError represents an error for unsupported task types type UnsupportedTaskTypeError struct { TaskType types.TaskType diff --git a/weed/worker/tasks/vacuum/detection.go b/weed/worker/tasks/vacuum/detection.go index 7b5a1baf0..23f82ad34 100644 --- a/weed/worker/tasks/vacuum/detection.go +++ b/weed/worker/tasks/vacuum/detection.go @@ -4,6 +4,7 @@ import ( "time" "github.com/seaweedfs/seaweedfs/weed/glog" + "github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" "github.com/seaweedfs/seaweedfs/weed/worker/tasks/base" "github.com/seaweedfs/seaweedfs/weed/worker/types" ) @@ -39,6 +40,9 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI Reason: "Volume has excessive garbage requiring vacuum", ScheduleAt: time.Now(), } + + // Create typed parameters for vacuum task + result.TypedParams = createVacuumTaskParams(result, metric, vacuumConfig) results = append(results, result) } else { // Debug why volume was not selected @@ -74,39 +78,36 @@ func Detection(metrics []*types.VolumeHealthMetrics, clusterInfo *types.ClusterI return results, nil } -// Scheduling implements the scheduling logic for vacuum tasks -func Scheduling(task *types.Task, runningTasks []*types.Task, availableWorkers []*types.Worker, config base.TaskConfig) bool { - vacuumConfig := config.(*Config) - - // Count running vacuum tasks - runningVacuumCount := 0 - for _, runningTask := range runningTasks { - if runningTask.Type == types.TaskTypeVacuum { - runningVacuumCount++ - } - } - - // Check concurrency limit - if runningVacuumCount >= vacuumConfig.MaxConcurrent { - return false +// createVacuumTaskParams creates typed parameters for vacuum tasks +// This function is moved from MaintenanceIntegration.createVacuumTaskParams to the detection logic +func createVacuumTaskParams(task *types.TaskDetectionResult, metric *types.VolumeHealthMetrics, vacuumConfig *Config) *worker_pb.TaskParams { + // Use configured values or defaults + 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 } - // Check for available workers with vacuum capability - for _, worker := range availableWorkers { - if worker.CurrentLoad < worker.MaxConcurrent { - for _, capability := range worker.Capabilities { - if capability == types.TaskTypeVacuum { - return true - } - } - } + // Create typed protobuf parameters + return &worker_pb.TaskParams{ + VolumeId: task.VolumeID, + Server: task.Server, + Collection: task.Collection, + VolumeSize: metric.Size, // Store original volume size for tracking changes + TaskParams: &worker_pb.TaskParams_VacuumParams{ + VacuumParams: &worker_pb.VacuumTaskParams{ + GarbageThreshold: garbageThreshold, + ForceVacuum: false, + BatchSize: batchSize, + WorkingDir: workingDir, + VerifyChecksum: verifyChecksum, + }, + }, } - - return false -} - -// CreateTask creates a new vacuum task instance -func CreateTask(params types.TaskParams) (types.TaskInterface, error) { - // Create and return the vacuum task using existing Task type - return NewTask(params.Server, params.VolumeID), nil } diff --git a/weed/worker/tasks/vacuum/monitoring.go b/weed/worker/tasks/vacuum/monitoring.go new file mode 100644 index 000000000..c7dfd673e --- /dev/null +++ b/weed/worker/tasks/vacuum/monitoring.go @@ -0,0 +1,151 @@ +package vacuum + +import ( + "sync" + "time" +) + +// VacuumMetrics contains vacuum-specific monitoring data +type VacuumMetrics struct { + // Execution metrics + VolumesVacuumed int64 `json:"volumes_vacuumed"` + TotalSpaceReclaimed int64 `json:"total_space_reclaimed"` + TotalFilesProcessed int64 `json:"total_files_processed"` + TotalGarbageCollected int64 `json:"total_garbage_collected"` + LastVacuumTime time.Time `json:"last_vacuum_time"` + + // Performance metrics + AverageVacuumTime int64 `json:"average_vacuum_time_seconds"` + AverageGarbageRatio float64 `json:"average_garbage_ratio"` + SuccessfulOperations int64 `json:"successful_operations"` + FailedOperations int64 `json:"failed_operations"` + + // Current task metrics + CurrentGarbageRatio float64 `json:"current_garbage_ratio"` + VolumesPendingVacuum int `json:"volumes_pending_vacuum"` + + mutex sync.RWMutex +} + +// NewVacuumMetrics creates a new vacuum metrics instance +func NewVacuumMetrics() *VacuumMetrics { + return &VacuumMetrics{ + LastVacuumTime: time.Now(), + } +} + +// RecordVolumeVacuumed records a successful volume vacuum operation +func (m *VacuumMetrics) RecordVolumeVacuumed(spaceReclaimed int64, filesProcessed int64, garbageCollected int64, vacuumTime time.Duration, garbageRatio float64) { + m.mutex.Lock() + defer m.mutex.Unlock() + + m.VolumesVacuumed++ + m.TotalSpaceReclaimed += spaceReclaimed + m.TotalFilesProcessed += filesProcessed + m.TotalGarbageCollected += garbageCollected + m.SuccessfulOperations++ + m.LastVacuumTime = time.Now() + + // Update average vacuum time + if m.AverageVacuumTime == 0 { + m.AverageVacuumTime = int64(vacuumTime.Seconds()) + } else { + // Exponential moving average + newTime := int64(vacuumTime.Seconds()) + m.AverageVacuumTime = (m.AverageVacuumTime*4 + newTime) / 5 + } + + // Update average garbage ratio + if m.AverageGarbageRatio == 0 { + m.AverageGarbageRatio = garbageRatio + } else { + // Exponential moving average + m.AverageGarbageRatio = 0.8*m.AverageGarbageRatio + 0.2*garbageRatio + } +} + +// RecordFailure records a failed vacuum operation +func (m *VacuumMetrics) RecordFailure() { + m.mutex.Lock() + defer m.mutex.Unlock() + + m.FailedOperations++ +} + +// UpdateCurrentGarbageRatio updates the current volume's garbage ratio +func (m *VacuumMetrics) UpdateCurrentGarbageRatio(ratio float64) { + m.mutex.Lock() + defer m.mutex.Unlock() + + m.CurrentGarbageRatio = ratio +} + +// SetVolumesPendingVacuum sets the number of volumes pending vacuum +func (m *VacuumMetrics) SetVolumesPendingVacuum(count int) { + m.mutex.Lock() + defer m.mutex.Unlock() + + m.VolumesPendingVacuum = count +} + +// GetMetrics returns a copy of the current metrics (without the mutex) +func (m *VacuumMetrics) GetMetrics() VacuumMetrics { + m.mutex.RLock() + defer m.mutex.RUnlock() + + // Create a copy without the mutex to avoid copying lock value + return VacuumMetrics{ + VolumesVacuumed: m.VolumesVacuumed, + TotalSpaceReclaimed: m.TotalSpaceReclaimed, + TotalFilesProcessed: m.TotalFilesProcessed, + TotalGarbageCollected: m.TotalGarbageCollected, + LastVacuumTime: m.LastVacuumTime, + AverageVacuumTime: m.AverageVacuumTime, + AverageGarbageRatio: m.AverageGarbageRatio, + SuccessfulOperations: m.SuccessfulOperations, + FailedOperations: m.FailedOperations, + CurrentGarbageRatio: m.CurrentGarbageRatio, + VolumesPendingVacuum: m.VolumesPendingVacuum, + } +} + +// GetSuccessRate returns the success rate as a percentage +func (m *VacuumMetrics) GetSuccessRate() float64 { + m.mutex.RLock() + defer m.mutex.RUnlock() + + total := m.SuccessfulOperations + m.FailedOperations + if total == 0 { + return 100.0 + } + return float64(m.SuccessfulOperations) / float64(total) * 100.0 +} + +// GetAverageSpaceReclaimed returns the average space reclaimed per volume +func (m *VacuumMetrics) GetAverageSpaceReclaimed() float64 { + m.mutex.RLock() + defer m.mutex.RUnlock() + + if m.VolumesVacuumed == 0 { + return 0 + } + return float64(m.TotalSpaceReclaimed) / float64(m.VolumesVacuumed) +} + +// Reset resets all metrics to zero +func (m *VacuumMetrics) Reset() { + m.mutex.Lock() + defer m.mutex.Unlock() + + *m = VacuumMetrics{ + LastVacuumTime: time.Now(), + } +} + +// Global metrics instance for vacuum tasks +var globalVacuumMetrics = NewVacuumMetrics() + +// GetGlobalVacuumMetrics returns the global vacuum metrics instance +func GetGlobalVacuumMetrics() *VacuumMetrics { + return globalVacuumMetrics +} diff --git a/weed/worker/tasks/vacuum/vacuum_register.go b/weed/worker/tasks/vacuum/register.go similarity index 82% rename from weed/worker/tasks/vacuum/vacuum_register.go rename to weed/worker/tasks/vacuum/register.go index d660c9d42..66d94d28e 100644 --- a/weed/worker/tasks/vacuum/vacuum_register.go +++ b/weed/worker/tasks/vacuum/register.go @@ -5,6 +5,7 @@ import ( "time" "github.com/seaweedfs/seaweedfs/weed/glog" + "github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" "github.com/seaweedfs/seaweedfs/weed/worker/tasks" "github.com/seaweedfs/seaweedfs/weed/worker/tasks/base" "github.com/seaweedfs/seaweedfs/weed/worker/types" @@ -35,9 +36,19 @@ func RegisterVacuumTask() { Icon: "fas fa-broom text-primary", Capabilities: []string{"vacuum", "storage"}, - Config: config, - ConfigSpec: GetConfigSpec(), - CreateTask: CreateTask, + Config: config, + ConfigSpec: GetConfigSpec(), + CreateTask: func(params *worker_pb.TaskParams) (types.Task, error) { + if params == nil { + return nil, fmt.Errorf("task parameters are required") + } + return NewVacuumTask( + fmt.Sprintf("vacuum-%d", params.VolumeId), + params.Server, + params.VolumeId, + params.Collection, + ), nil + }, DetectionFunc: Detection, ScanInterval: 2 * time.Hour, SchedulingFunc: Scheduling, diff --git a/weed/worker/tasks/vacuum/scheduling.go b/weed/worker/tasks/vacuum/scheduling.go new file mode 100644 index 000000000..c44724eb9 --- /dev/null +++ b/weed/worker/tasks/vacuum/scheduling.go @@ -0,0 +1,37 @@ +package vacuum + +import ( + "github.com/seaweedfs/seaweedfs/weed/worker/tasks/base" + "github.com/seaweedfs/seaweedfs/weed/worker/types" +) + +// Scheduling implements the scheduling logic for vacuum tasks +func Scheduling(task *types.TaskInput, runningTasks []*types.TaskInput, availableWorkers []*types.WorkerData, config base.TaskConfig) bool { + vacuumConfig := config.(*Config) + + // Count running vacuum tasks + runningVacuumCount := 0 + for _, runningTask := range runningTasks { + if runningTask.Type == types.TaskTypeVacuum { + runningVacuumCount++ + } + } + + // Check concurrency limit + if runningVacuumCount >= vacuumConfig.MaxConcurrent { + return false + } + + // Check for available workers with vacuum capability + for _, worker := range availableWorkers { + if worker.CurrentLoad < worker.MaxConcurrent { + for _, capability := range worker.Capabilities { + if capability == types.TaskTypeVacuum { + return true + } + } + } + } + + return false +} diff --git a/weed/worker/tasks/vacuum/vacuum.go b/weed/worker/tasks/vacuum/vacuum.go deleted file mode 100644 index 9cd254958..000000000 --- a/weed/worker/tasks/vacuum/vacuum.go +++ /dev/null @@ -1,214 +0,0 @@ -package vacuum - -import ( - "context" - "fmt" - "io" - "time" - - "github.com/seaweedfs/seaweedfs/weed/pb" - "github.com/seaweedfs/seaweedfs/weed/pb/volume_server_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" -) - -// Task implements vacuum operation to reclaim disk space -type Task struct { - *tasks.BaseTask - server string - volumeID uint32 - garbageThreshold float64 -} - -// NewTask creates a new vacuum task instance -func NewTask(server string, volumeID uint32) *Task { - task := &Task{ - BaseTask: tasks.NewBaseTask(types.TaskTypeVacuum), - server: server, - volumeID: volumeID, - garbageThreshold: 0.3, // Default 30% threshold - } - return task -} - -// Execute performs the vacuum operation -func (t *Task) Execute(params types.TaskParams) error { - // Use BaseTask.ExecuteTask to handle logging initialization - return t.ExecuteTask(context.Background(), params, t.executeImpl) -} - -// executeImpl is the actual vacuum implementation -func (t *Task) executeImpl(ctx context.Context, params types.TaskParams) error { - t.LogInfo("Starting vacuum for volume %d on server %s", t.volumeID, t.server) - - // Parse garbage threshold from typed parameters - if params.TypedParams != nil { - if vacuumParams := params.TypedParams.GetVacuumParams(); vacuumParams != nil { - t.garbageThreshold = vacuumParams.GarbageThreshold - t.LogWithFields("INFO", "Using garbage threshold from parameters", map[string]interface{}{ - "threshold": t.garbageThreshold, - }) - } - } - - // Convert server address to gRPC address and use proper dial option - grpcAddress := pb.ServerToGrpcAddress(t.server) - var dialOpt grpc.DialOption = grpc.WithTransportCredentials(insecure.NewCredentials()) - if params.GrpcDialOption != nil { - dialOpt = params.GrpcDialOption - } - - conn, err := grpc.NewClient(grpcAddress, dialOpt) - if err != nil { - t.LogError("Failed to connect to volume server %s: %v", t.server, err) - return fmt.Errorf("failed to connect to volume server %s: %v", t.server, err) - } - defer conn.Close() - - client := volume_server_pb.NewVolumeServerClient(conn) - - // Step 1: Check vacuum eligibility - t.SetProgress(10.0) - t.LogDebug("Checking vacuum eligibility for volume %d", t.volumeID) - - checkResp, err := client.VacuumVolumeCheck(ctx, &volume_server_pb.VacuumVolumeCheckRequest{ - VolumeId: t.volumeID, - }) - if err != nil { - t.LogError("Vacuum check failed for volume %d: %v", t.volumeID, err) - return fmt.Errorf("vacuum check failed for volume %d: %v", t.volumeID, err) - } - - // Check if garbage ratio meets threshold - if checkResp.GarbageRatio < t.garbageThreshold { - t.LogWarning("Volume %d garbage ratio %.2f%% is below threshold %.2f%%, skipping vacuum", - t.volumeID, checkResp.GarbageRatio*100, t.garbageThreshold*100) - return fmt.Errorf("volume %d garbage ratio %.2f%% is below threshold %.2f%%, skipping vacuum", - t.volumeID, checkResp.GarbageRatio*100, t.garbageThreshold*100) - } - - t.LogWithFields("INFO", "Volume eligible for vacuum", map[string]interface{}{ - "volume_id": t.volumeID, - "garbage_ratio": checkResp.GarbageRatio, - "threshold": t.garbageThreshold, - "garbage_percent": checkResp.GarbageRatio * 100, - }) - - // Step 2: Compact volume - t.SetProgress(30.0) - t.LogInfo("Starting compact for volume %d", t.volumeID) - - compactStream, err := client.VacuumVolumeCompact(ctx, &volume_server_pb.VacuumVolumeCompactRequest{ - VolumeId: t.volumeID, - }) - if err != nil { - t.LogError("Vacuum compact failed for volume %d: %v", t.volumeID, err) - return fmt.Errorf("vacuum compact failed for volume %d: %v", t.volumeID, err) - } - - // Process compact stream and track progress - var processedBytes int64 - var totalBytes int64 - - for { - resp, err := compactStream.Recv() - if err != nil { - if err == io.EOF { - break - } - t.LogError("Vacuum compact stream error for volume %d: %v", t.volumeID, err) - return fmt.Errorf("vacuum compact stream error for volume %d: %v", t.volumeID, err) - } - - processedBytes = resp.ProcessedBytes - if resp.LoadAvg_1M > 0 { - totalBytes = int64(resp.LoadAvg_1M) // This is a rough approximation - } - - // Update progress based on processed bytes (30% to 70% of total progress) - if totalBytes > 0 { - compactProgress := float64(processedBytes) / float64(totalBytes) - if compactProgress > 1.0 { - compactProgress = 1.0 - } - progress := 30.0 + (compactProgress * 40.0) // 30% to 70% - t.SetProgress(progress) - } - - t.LogWithFields("DEBUG", "Volume compact progress", map[string]interface{}{ - "volume_id": t.volumeID, - "processed_bytes": processedBytes, - "total_bytes": totalBytes, - "compact_progress": fmt.Sprintf("%.1f%%", (float64(processedBytes)/float64(totalBytes))*100), - }) - } - - // Step 3: Commit vacuum changes - t.SetProgress(80.0) - t.LogInfo("Committing vacuum for volume %d", t.volumeID) - - commitResp, err := client.VacuumVolumeCommit(ctx, &volume_server_pb.VacuumVolumeCommitRequest{ - VolumeId: t.volumeID, - }) - if err != nil { - t.LogError("Vacuum commit failed for volume %d: %v", t.volumeID, err) - return fmt.Errorf("vacuum commit failed for volume %d: %v", t.volumeID, err) - } - - // Step 4: Cleanup temporary files - t.SetProgress(90.0) - t.LogInfo("Cleaning up vacuum files for volume %d", t.volumeID) - - _, err = client.VacuumVolumeCleanup(ctx, &volume_server_pb.VacuumVolumeCleanupRequest{ - VolumeId: t.volumeID, - }) - if err != nil { - // Log warning but don't fail the task - t.LogWarning("Vacuum cleanup warning for volume %d: %v", t.volumeID, err) - } - - t.SetProgress(100.0) - - newVolumeSize := commitResp.VolumeSize - t.LogWithFields("INFO", "Successfully completed vacuum", map[string]interface{}{ - "volume_id": t.volumeID, - "server": t.server, - "new_volume_size": newVolumeSize, - "garbage_reclaimed": true, - }) - - return nil -} - -// Validate validates the task parameters -func (t *Task) Validate(params types.TaskParams) error { - if params.VolumeID == 0 { - return fmt.Errorf("volume_id is required") - } - if params.Server == "" { - return fmt.Errorf("server is required") - } - return nil -} - -// EstimateTime estimates the time needed for the task -func (t *Task) EstimateTime(params types.TaskParams) time.Duration { - // Base time for vacuum operations - varies by volume size and garbage ratio - // Typically vacuum is faster than EC encoding - baseTime := 5 * time.Minute - - // Use default estimation since volume size is not available in typed params - return baseTime -} - -// GetProgress returns the current progress -func (t *Task) GetProgress() float64 { - return t.BaseTask.GetProgress() -} - -// Cancel cancels the task -func (t *Task) Cancel() error { - return t.BaseTask.Cancel() -} diff --git a/weed/worker/tasks/vacuum/vacuum_task.go b/weed/worker/tasks/vacuum/vacuum_task.go new file mode 100644 index 000000000..005f5a681 --- /dev/null +++ b/weed/worker/tasks/vacuum/vacuum_task.go @@ -0,0 +1,236 @@ +package vacuum + +import ( + "context" + "fmt" + "io" + "time" + + "github.com/seaweedfs/seaweedfs/weed/glog" + "github.com/seaweedfs/seaweedfs/weed/operation" + "github.com/seaweedfs/seaweedfs/weed/pb" + "github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb" + "github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" + "github.com/seaweedfs/seaweedfs/weed/worker/types" + "github.com/seaweedfs/seaweedfs/weed/worker/types/base" + "google.golang.org/grpc" +) + +// VacuumTask implements the Task interface +type VacuumTask struct { + *base.BaseTask + server string + volumeID uint32 + collection string + garbageThreshold float64 + progress float64 +} + +// NewVacuumTask creates a new unified vacuum task instance +func NewVacuumTask(id string, server string, volumeID uint32, collection string) *VacuumTask { + return &VacuumTask{ + BaseTask: base.NewBaseTask(id, types.TaskTypeVacuum), + server: server, + volumeID: volumeID, + collection: collection, + garbageThreshold: 0.3, // Default 30% threshold + } +} + +// Execute implements the UnifiedTask interface +func (t *VacuumTask) Execute(ctx context.Context, params *worker_pb.TaskParams) error { + if params == nil { + return fmt.Errorf("task parameters are required") + } + + vacuumParams := params.GetVacuumParams() + if vacuumParams == nil { + return fmt.Errorf("vacuum parameters are required") + } + + t.garbageThreshold = vacuumParams.GarbageThreshold + + t.GetLogger().WithFields(map[string]interface{}{ + "volume_id": t.volumeID, + "server": t.server, + "collection": t.collection, + "garbage_threshold": t.garbageThreshold, + }).Info("Starting vacuum task") + + // Step 1: Check volume status and garbage ratio + t.ReportProgress(10.0) + t.GetLogger().Info("Checking volume status") + eligible, currentGarbageRatio, err := t.checkVacuumEligibility() + if err != nil { + return fmt.Errorf("failed to check vacuum eligibility: %v", err) + } + + if !eligible { + t.GetLogger().WithFields(map[string]interface{}{ + "current_garbage_ratio": currentGarbageRatio, + "required_threshold": t.garbageThreshold, + }).Info("Volume does not meet vacuum criteria, skipping") + t.ReportProgress(100.0) + return nil + } + + // Step 2: Perform vacuum operation + t.ReportProgress(50.0) + t.GetLogger().WithFields(map[string]interface{}{ + "garbage_ratio": currentGarbageRatio, + "threshold": t.garbageThreshold, + }).Info("Performing vacuum operation") + + if err := t.performVacuum(); err != nil { + return fmt.Errorf("failed to perform vacuum: %v", err) + } + + // Step 3: Verify vacuum results + t.ReportProgress(90.0) + t.GetLogger().Info("Verifying vacuum results") + if err := t.verifyVacuumResults(); err != nil { + glog.Warningf("Vacuum verification failed: %v", err) + // Don't fail the task - vacuum operation itself succeeded + } + + t.ReportProgress(100.0) + glog.Infof("Vacuum task completed successfully: volume %d from %s (garbage ratio was %.2f%%)", + t.volumeID, t.server, currentGarbageRatio*100) + return nil +} + +// Validate implements the UnifiedTask interface +func (t *VacuumTask) Validate(params *worker_pb.TaskParams) error { + if params == nil { + return fmt.Errorf("task parameters are required") + } + + vacuumParams := params.GetVacuumParams() + if vacuumParams == nil { + return fmt.Errorf("vacuum parameters are required") + } + + if params.VolumeId != t.volumeID { + return fmt.Errorf("volume ID mismatch: expected %d, got %d", t.volumeID, params.VolumeId) + } + + if params.Server != t.server { + return fmt.Errorf("source server mismatch: expected %s, got %s", t.server, params.Server) + } + + if vacuumParams.GarbageThreshold < 0 || vacuumParams.GarbageThreshold > 1.0 { + return fmt.Errorf("invalid garbage threshold: %f (must be between 0.0 and 1.0)", vacuumParams.GarbageThreshold) + } + + return nil +} + +// EstimateTime implements the UnifiedTask interface +func (t *VacuumTask) EstimateTime(params *worker_pb.TaskParams) time.Duration { + // Basic estimate based on simulated steps + return 14 * time.Second // Sum of all step durations +} + +// GetProgress returns current progress +func (t *VacuumTask) GetProgress() float64 { + return t.progress +} + +// Helper methods for real vacuum operations + +// checkVacuumEligibility checks if the volume meets vacuum criteria +func (t *VacuumTask) checkVacuumEligibility() (bool, float64, error) { + var garbageRatio float64 + + err := operation.WithVolumeServerClient(false, pb.ServerAddress(t.server), grpc.WithInsecure(), + func(client volume_server_pb.VolumeServerClient) error { + resp, err := client.VacuumVolumeCheck(context.Background(), &volume_server_pb.VacuumVolumeCheckRequest{ + VolumeId: t.volumeID, + }) + if err != nil { + return fmt.Errorf("failed to check volume vacuum status: %v", err) + } + + garbageRatio = resp.GarbageRatio + + return nil + }) + + if err != nil { + return false, 0, err + } + + eligible := garbageRatio >= t.garbageThreshold + glog.V(1).Infof("Volume %d garbage ratio: %.2f%%, threshold: %.2f%%, eligible: %v", + t.volumeID, garbageRatio*100, t.garbageThreshold*100, eligible) + + return eligible, garbageRatio, nil +} + +// performVacuum executes the actual vacuum operation +func (t *VacuumTask) performVacuum() error { + return operation.WithVolumeServerClient(false, pb.ServerAddress(t.server), grpc.WithInsecure(), + func(client volume_server_pb.VolumeServerClient) error { + // Step 1: Compact the volume + t.GetLogger().Info("Compacting volume") + stream, err := client.VacuumVolumeCompact(context.Background(), &volume_server_pb.VacuumVolumeCompactRequest{ + VolumeId: t.volumeID, + }) + if err != nil { + return fmt.Errorf("vacuum compact failed: %v", err) + } + + // Read compact progress + for { + resp, recvErr := stream.Recv() + if recvErr != nil { + if recvErr == io.EOF { + break + } + return fmt.Errorf("vacuum compact stream error: %v", recvErr) + } + glog.V(2).Infof("Volume %d compact progress: %d bytes processed", t.volumeID, resp.ProcessedBytes) + } + + // Step 2: Commit the vacuum + t.GetLogger().Info("Committing vacuum operation") + _, err = client.VacuumVolumeCommit(context.Background(), &volume_server_pb.VacuumVolumeCommitRequest{ + VolumeId: t.volumeID, + }) + if err != nil { + return fmt.Errorf("vacuum commit failed: %v", err) + } + + // Step 3: Cleanup old files + t.GetLogger().Info("Cleaning up vacuum files") + _, err = client.VacuumVolumeCleanup(context.Background(), &volume_server_pb.VacuumVolumeCleanupRequest{ + VolumeId: t.volumeID, + }) + if err != nil { + return fmt.Errorf("vacuum cleanup failed: %v", err) + } + + glog.V(1).Infof("Volume %d vacuum operation completed successfully", t.volumeID) + return nil + }) +} + +// verifyVacuumResults checks the volume status after vacuum +func (t *VacuumTask) verifyVacuumResults() error { + return operation.WithVolumeServerClient(false, pb.ServerAddress(t.server), grpc.WithInsecure(), + func(client volume_server_pb.VolumeServerClient) error { + resp, err := client.VacuumVolumeCheck(context.Background(), &volume_server_pb.VacuumVolumeCheckRequest{ + VolumeId: t.volumeID, + }) + if err != nil { + return fmt.Errorf("failed to verify vacuum results: %v", err) + } + + postVacuumGarbageRatio := resp.GarbageRatio + + glog.V(1).Infof("Volume %d post-vacuum garbage ratio: %.2f%%", + t.volumeID, postVacuumGarbageRatio*100) + + return nil + }) +} diff --git a/weed/worker/types/base/task.go b/weed/worker/types/base/task.go new file mode 100644 index 000000000..5403f8ae9 --- /dev/null +++ b/weed/worker/types/base/task.go @@ -0,0 +1,99 @@ +package base + +import ( + "context" + "time" + + "github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" + "github.com/seaweedfs/seaweedfs/weed/worker/types" +) + +// BaseTask provides common task functionality +type BaseTask struct { + id string + taskType types.TaskType + progressCallback func(float64) + logger types.Logger + cancelled bool +} + +// NewBaseTask creates a new base task +func NewBaseTask(id string, taskType types.TaskType) *BaseTask { + return &BaseTask{ + id: id, + taskType: taskType, + logger: &types.GlogFallbackLogger{}, // Default fallback logger + } +} + +// ID returns the task ID +func (t *BaseTask) ID() string { + return t.id +} + +// Type returns the task type +func (t *BaseTask) Type() types.TaskType { + return t.taskType +} + +// SetProgressCallback sets the progress callback +func (t *BaseTask) SetProgressCallback(callback func(float64)) { + t.progressCallback = callback +} + +// ReportProgress reports current progress through the callback +func (t *BaseTask) ReportProgress(progress float64) { + if t.progressCallback != nil { + t.progressCallback(progress) + } +} + +// GetProgress returns current progress +func (t *BaseTask) GetProgress() float64 { + // Subclasses should override this + return 0 +} + +// Cancel marks the task as cancelled +func (t *BaseTask) Cancel() error { + t.cancelled = true + return nil +} + +// IsCancellable returns true if the task can be cancelled +func (t *BaseTask) IsCancellable() bool { + return true +} + +// IsCancelled returns true if the task has been cancelled +func (t *BaseTask) IsCancelled() bool { + return t.cancelled +} + +// SetLogger sets the task logger +func (t *BaseTask) SetLogger(logger types.Logger) { + t.logger = logger +} + +// GetLogger returns the task logger +func (t *BaseTask) GetLogger() types.Logger { + return t.logger +} + +// Execute implements the Task interface +func (t *BaseTask) Execute(ctx context.Context, params *worker_pb.TaskParams) error { + // Subclasses must implement this + return nil +} + +// Validate implements the UnifiedTask interface +func (t *BaseTask) Validate(params *worker_pb.TaskParams) error { + // Subclasses must implement this + return nil +} + +// EstimateTime implements the UnifiedTask interface +func (t *BaseTask) EstimateTime(params *worker_pb.TaskParams) time.Duration { + // Subclasses must implement this + return 0 +} diff --git a/weed/worker/types/data_types.go b/weed/worker/types/data_types.go index c6ba62a18..203cbfadb 100644 --- a/weed/worker/types/data_types.go +++ b/weed/worker/types/data_types.go @@ -2,14 +2,17 @@ package types import ( "time" + + "github.com/seaweedfs/seaweedfs/weed/admin/topology" ) // ClusterInfo contains cluster information for task detection type ClusterInfo struct { - Servers []*VolumeServerInfo - TotalVolumes int - TotalServers int - LastUpdated time.Time + Servers []*VolumeServerInfo + TotalVolumes int + TotalServers int + LastUpdated time.Time + ActiveTopology *topology.ActiveTopology // Added for destination planning in detection } // VolumeHealthMetrics contains health information about a volume (simplified) diff --git a/weed/worker/types/factory.go b/weed/worker/types/factory.go new file mode 100644 index 000000000..f61694239 --- /dev/null +++ b/weed/worker/types/factory.go @@ -0,0 +1,52 @@ +package types + +// This file contains the unified factory interfaces. + +import ( + "context" + "time" + + "github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" +) + +// Factory defines a generic factory interface +type Factory[T any, C any] interface { + // Create new instance with context and config + Create(ctx context.Context, config C) (T, error) + + // Metadata + Type() string + Description() string + Capabilities() []string +} + +// TaskFactory creates new task instances +type TaskFactory interface { + Create(params *worker_pb.TaskParams) (Task, error) + Type() string + Description() string + Capabilities() []string +} + +// TaskCreationConfig defines task creation configuration +type TaskCreationConfig struct { + ID string + Type TaskType + Server string + Collection string + VolumeID uint32 + Logger Logger +} + +// WorkerCreationConfig encapsulates all worker configuration +type WorkerCreationConfig struct { + ID string + Capabilities []TaskType + MaxConcurrent int + HeartbeatInterval time.Duration + TaskRequestInterval time.Duration + LoggerFactory LoggerFactory +} + +// WorkerFactory creates new worker instances +type WorkerFactory = Factory[Worker, WorkerConfig] diff --git a/weed/worker/types/task.go b/weed/worker/types/task.go new file mode 100644 index 000000000..2c9ed7f8a --- /dev/null +++ b/weed/worker/types/task.go @@ -0,0 +1,189 @@ +package types + +// This file contains the new unified task interfaces that will replace +// the existing TaskInterface and TypedTaskInterface. + +import ( + "context" + "time" + + "github.com/seaweedfs/seaweedfs/weed/glog" + "github.com/seaweedfs/seaweedfs/weed/pb/worker_pb" +) + +// Task defines the core task interface that all tasks must implement +type Task interface { + // Core identity + ID() string + Type() TaskType + + // Execution + Execute(ctx context.Context, params *worker_pb.TaskParams) error + Validate(params *worker_pb.TaskParams) error + EstimateTime(params *worker_pb.TaskParams) time.Duration + + // Control + Cancel() error + IsCancellable() bool + + // Progress + GetProgress() float64 + SetProgressCallback(func(float64)) +} + +// TaskWithLogging extends Task with logging capabilities +type TaskWithLogging interface { + Task + Logger +} + +// Logger defines standard logging interface +type Logger interface { + Info(msg string, args ...interface{}) + Warning(msg string, args ...interface{}) + Error(msg string, args ...interface{}) + Debug(msg string, args ...interface{}) + WithFields(fields map[string]interface{}) Logger +} + +// NoOpLogger is a logger that does nothing (silent) +type NoOpLogger struct{} + +func (l *NoOpLogger) Info(msg string, args ...interface{}) {} +func (l *NoOpLogger) Warning(msg string, args ...interface{}) {} +func (l *NoOpLogger) Error(msg string, args ...interface{}) {} +func (l *NoOpLogger) Debug(msg string, args ...interface{}) {} +func (l *NoOpLogger) WithFields(fields map[string]interface{}) Logger { + return l // Return self since we're doing nothing anyway +} + +// GlogFallbackLogger is a logger that falls back to glog +type GlogFallbackLogger struct{} + +func (l *GlogFallbackLogger) Info(msg string, args ...interface{}) { + if len(args) > 0 { + glog.Infof(msg, args...) + } else { + glog.Info(msg) + } +} + +func (l *GlogFallbackLogger) Warning(msg string, args ...interface{}) { + if len(args) > 0 { + glog.Warningf(msg, args...) + } else { + glog.Warning(msg) + } +} + +func (l *GlogFallbackLogger) Error(msg string, args ...interface{}) { + if len(args) > 0 { + glog.Errorf(msg, args...) + } else { + glog.Error(msg) + } +} + +func (l *GlogFallbackLogger) Debug(msg string, args ...interface{}) { + if len(args) > 0 { + glog.V(1).Infof(msg, args...) + } else { + glog.V(1).Info(msg) + } +} + +func (l *GlogFallbackLogger) WithFields(fields map[string]interface{}) Logger { + // For glog fallback, we'll just return self and ignore fields for simplicity + // A more sophisticated implementation could format the fields into the message + return l +} + +// LogLevel represents logging severity levels +type LogLevel int + +const ( + LogLevelDebug LogLevel = iota + LogLevelInfo + LogLevelWarning + LogLevelError +) + +// LoggerConfig defines logger configuration +type LoggerConfig struct { + MinLevel LogLevel + MaxSize int64 + MaxFiles int + Directory string + ServiceName string + EnableJSON bool +} + +// LoggerFactory creates configured loggers +type LoggerFactory interface { + CreateLogger(ctx context.Context, config LoggerConfig) (Logger, error) +} + +// BaseTask provides common task functionality +type UnifiedBaseTask struct { + id string + taskType TaskType + progressCallback func(float64) + logger Logger + cancelled bool +} + +// NewBaseTask creates a new base task +func NewUnifiedBaseTask(id string, taskType TaskType) *UnifiedBaseTask { + return &UnifiedBaseTask{ + id: id, + taskType: taskType, + } +} + +// ID returns the task ID +func (t *UnifiedBaseTask) ID() string { + return t.id +} + +// Type returns the task type +func (t *UnifiedBaseTask) Type() TaskType { + return t.taskType +} + +// SetProgressCallback sets the progress callback +func (t *UnifiedBaseTask) SetProgressCallback(callback func(float64)) { + t.progressCallback = callback +} + +// ReportProgress reports current progress through the callback +func (t *UnifiedBaseTask) ReportProgress(progress float64) { + if t.progressCallback != nil { + t.progressCallback(progress) + } +} + +// Cancel marks the task as cancelled +func (t *UnifiedBaseTask) Cancel() error { + t.cancelled = true + return nil +} + +// IsCancellable returns true if the task can be cancelled +func (t *UnifiedBaseTask) IsCancellable() bool { + return true +} + +// IsCancelled returns true if the task has been cancelled +func (t *UnifiedBaseTask) IsCancelled() bool { + return t.cancelled +} + +// SetLogger sets the task logger +func (t *UnifiedBaseTask) SetLogger(logger Logger) { + t.logger = logger +} + +// GetLogger returns the task logger +func (t *UnifiedBaseTask) GetLogger() Logger { + return t.logger +} diff --git a/weed/worker/types/task_scheduler.go b/weed/worker/types/task_scheduler.go index 958bf892a..b294417bc 100644 --- a/weed/worker/types/task_scheduler.go +++ b/weed/worker/types/task_scheduler.go @@ -8,10 +8,10 @@ type TaskScheduler interface { GetTaskType() TaskType // CanScheduleNow determines if a task can be scheduled now - CanScheduleNow(task *Task, runningTasks []*Task, availableWorkers []*Worker) bool + CanScheduleNow(task *TaskInput, runningTasks []*TaskInput, availableWorkers []*WorkerData) bool // GetPriority returns the priority for tasks of this type - GetPriority(task *Task) TaskPriority + GetPriority(task *TaskInput) TaskPriority // GetMaxConcurrent returns the maximum concurrent tasks of this type GetMaxConcurrent() int diff --git a/weed/worker/types/task_types.go b/weed/worker/types/task_types.go index dc454c211..ed7fc8f07 100644 --- a/weed/worker/types/task_types.go +++ b/weed/worker/types/task_types.go @@ -40,8 +40,8 @@ const ( TaskPriorityCritical TaskPriority = "critical" ) -// Task represents a maintenance task -type Task struct { +// TaskInput represents a maintenance task data +type TaskInput struct { ID string `json:"id"` Type TaskType `json:"type"` Status TaskStatus `json:"status"` diff --git a/weed/worker/types/worker.go b/weed/worker/types/worker.go new file mode 100644 index 000000000..9db5ba2c4 --- /dev/null +++ b/weed/worker/types/worker.go @@ -0,0 +1,76 @@ +package types + +// This file contains the new unified worker interfaces that will replace +// the existing WorkerInterface. + +import ( + "context" +) + +// Worker defines core worker functionality +type Worker interface { + // Core operations + Start(ctx context.Context) error + Stop(ctx context.Context) error + HandleTask(ctx context.Context, task Task) error + + // Status + GetStatus() WorkerStatus + GetCapabilities() []TaskType + + // Configuration + Configure(config WorkerCreationConfig) error +} + +// BaseWorker provides common worker functionality +type BaseWorker struct { + id string + capabilities []TaskType + maxConcurrent int + currentTasks map[string]Task + logger Logger +} + +// NewBaseWorker creates a new base worker +func NewBaseWorker(id string) *BaseWorker { + return &BaseWorker{ + id: id, + currentTasks: make(map[string]Task), + } +} + +// Configure applies worker configuration +func (w *BaseWorker) Configure(config WorkerCreationConfig) error { + w.id = config.ID + w.capabilities = config.Capabilities + w.maxConcurrent = config.MaxConcurrent + + if config.LoggerFactory != nil { + logger, err := config.LoggerFactory.CreateLogger(context.Background(), LoggerConfig{ + ServiceName: "worker-" + w.id, + MinLevel: LogLevelInfo, + }) + if err != nil { + return err + } + w.logger = logger + } + + return nil +} + +// GetCapabilities returns worker capabilities +func (w *BaseWorker) GetCapabilities() []TaskType { + return w.capabilities +} + +// GetStatus returns current worker status +func (w *BaseWorker) GetStatus() WorkerStatus { + return WorkerStatus{ + WorkerID: w.id, + Status: "active", + Capabilities: w.capabilities, + MaxConcurrent: w.maxConcurrent, + CurrentLoad: len(w.currentTasks), + } +} diff --git a/weed/worker/types/worker_types.go b/weed/worker/types/worker_types.go index b9b13e6c9..f84aa87f7 100644 --- a/weed/worker/types/worker_types.go +++ b/weed/worker/types/worker_types.go @@ -4,13 +4,13 @@ import ( "time" ) -// Worker represents a maintenance worker instance -type Worker struct { +// WorkerData represents a maintenance worker instance data +type WorkerData struct { ID string `json:"id"` Address string `json:"address"` LastHeartbeat time.Time `json:"last_heartbeat"` Status string `json:"status"` // active, inactive, busy - CurrentTask *Task `json:"current_task,omitempty"` + CurrentTask *TaskInput `json:"current_task,omitempty"` Capabilities []TaskType `json:"capabilities"` MaxConcurrent int `json:"max_concurrent"` CurrentLoad int `json:"current_load"` @@ -24,7 +24,7 @@ type WorkerStatus struct { MaxConcurrent int `json:"max_concurrent"` CurrentLoad int `json:"current_load"` LastHeartbeat time.Time `json:"last_heartbeat"` - CurrentTasks []Task `json:"current_tasks"` + CurrentTasks []TaskInput `json:"current_tasks"` Uptime time.Duration `json:"uptime"` TasksCompleted int `json:"tasks_completed"` TasksFailed int `json:"tasks_failed"` @@ -32,9 +32,9 @@ type WorkerStatus struct { // WorkerDetailsData represents detailed worker information type WorkerDetailsData struct { - Worker *Worker `json:"worker"` - CurrentTasks []*Task `json:"current_tasks"` - RecentTasks []*Task `json:"recent_tasks"` + Worker *WorkerData `json:"worker"` + CurrentTasks []*TaskInput `json:"current_tasks"` + RecentTasks []*TaskInput `json:"recent_tasks"` Performance *WorkerPerformance `json:"performance"` LastUpdated time.Time `json:"last_updated"` } @@ -70,42 +70,3 @@ type WorkerSummary struct { TotalLoad int `json:"total_load"` MaxCapacity int `json:"max_capacity"` } - -// WorkerFactory creates worker instances -type WorkerFactory interface { - Create(config WorkerConfig) (WorkerInterface, error) - Type() string - Description() string -} - -// WorkerInterface defines the interface for all worker implementations -type WorkerInterface interface { - ID() string - Start() error - Stop() error - RegisterTask(taskType TaskType, factory TaskFactory) - GetCapabilities() []TaskType - GetStatus() WorkerStatus - HandleTask(task *Task) error - SetCapabilities(capabilities []TaskType) - SetMaxConcurrent(max int) - SetHeartbeatInterval(interval time.Duration) - SetTaskRequestInterval(interval time.Duration) -} - -// TaskFactory creates task instances -type TaskFactory interface { - Create(params TaskParams) (TaskInterface, error) - Capabilities() []string - Description() string -} - -// TaskInterface defines the interface for all task implementations -type TaskInterface interface { - Type() TaskType - Execute(params TaskParams) error - Validate(params TaskParams) error - EstimateTime(params TaskParams) time.Duration - GetProgress() float64 - Cancel() error -} diff --git a/weed/worker/worker.go b/weed/worker/worker.go index ff6b87808..2bc0e1e11 100644 --- a/weed/worker/worker.go +++ b/weed/worker/worker.go @@ -1,6 +1,7 @@ package worker import ( + "context" "crypto/rand" "fmt" "net" @@ -26,7 +27,7 @@ type Worker struct { id string config *types.WorkerConfig registry *tasks.TaskRegistry - currentTasks map[string]*types.Task + currentTasks map[string]*types.TaskInput adminClient AdminClient running bool stopChan chan struct{} @@ -43,9 +44,9 @@ type Worker struct { type AdminClient interface { Connect() error Disconnect() error - RegisterWorker(worker *types.Worker) error + RegisterWorker(worker *types.WorkerData) error SendHeartbeat(workerID string, status *types.WorkerStatus) error - RequestTask(workerID string, capabilities []types.TaskType) (*types.Task, error) + RequestTask(workerID string, capabilities []types.TaskType) (*types.TaskInput, error) CompleteTask(taskID string, success bool, errorMsg string) error CompleteTaskWithMetadata(taskID string, success bool, errorMsg string, metadata map[string]string) error UpdateTaskProgress(taskID string, progress float64) error @@ -139,8 +140,8 @@ func NewWorker(config *types.WorkerConfig) (*Worker, error) { return nil, fmt.Errorf("failed to generate or load worker ID: %w", err) } - // Use the global registry that already has all tasks registered - registry := tasks.GetGlobalRegistry() + // Use the global unified registry that already has all tasks registered + registry := tasks.GetGlobalTaskRegistry() // Initialize task log handler logDir := filepath.Join(config.BaseWorkingDir, "task_logs") @@ -150,13 +151,13 @@ func NewWorker(config *types.WorkerConfig) (*Worker, error) { id: workerID, config: config, registry: registry, - currentTasks: make(map[string]*types.Task), + currentTasks: make(map[string]*types.TaskInput), stopChan: make(chan struct{}), startTime: time.Now(), taskLogHandler: taskLogHandler, } - glog.V(1).Infof("Worker created with %d registered task types", len(registry.GetSupportedTypes())) + glog.V(1).Infof("Worker created with %d registered task types", len(registry.GetAll())) return worker, nil } @@ -194,7 +195,7 @@ func (w *Worker) Start() error { w.startTime = time.Now() // Prepare worker info for registration - workerInfo := &types.Worker{ + workerInfo := &types.WorkerData{ ID: w.id, Capabilities: w.config.Capabilities, MaxConcurrent: w.config.MaxConcurrent, @@ -293,7 +294,7 @@ func (w *Worker) GetStatus() types.WorkerStatus { w.mutex.RLock() defer w.mutex.RUnlock() - var currentTasks []types.Task + var currentTasks []types.TaskInput for _, task := range w.currentTasks { currentTasks = append(currentTasks, *task) } @@ -318,7 +319,7 @@ func (w *Worker) GetStatus() types.WorkerStatus { } // HandleTask handles a task execution -func (w *Worker) HandleTask(task *types.Task) error { +func (w *Worker) HandleTask(task *types.TaskInput) error { glog.V(1).Infof("Worker %s received task %s (type: %s, volume: %d)", w.id, task.ID, task.Type, task.VolumeID) @@ -370,7 +371,7 @@ func (w *Worker) SetAdminClient(client AdminClient) { } // executeTask executes a task -func (w *Worker) executeTask(task *types.Task) { +func (w *Worker) executeTask(task *types.TaskInput) { startTime := time.Now() defer func() { @@ -403,44 +404,35 @@ func (w *Worker) executeTask(task *types.Task) { return } - // Use typed task execution (all tasks should be typed) + // Use new task execution system with unified Task interface glog.V(1).Infof("Executing task %s with typed protobuf parameters", task.ID) - typedRegistry := types.GetGlobalTypedTaskRegistry() - typedTaskInstance, err := typedRegistry.CreateTypedTask(task.Type) - if err != nil { - w.completeTask(task.ID, false, fmt.Sprintf("typed task not available for %s: %v", task.Type, err)) - glog.Errorf("Worker %s failed to create typed task %s: %v", w.id, task.ID, err) - return - } + taskFactory := w.registry.Get(task.Type) + if taskFactory == nil { + w.completeTask(task.ID, false, fmt.Sprintf("task factory not available for %s: task type not found", task.Type)) + glog.Errorf("Worker %s failed to get task factory for %s type %v", w.id, task.ID, task.Type) - // Configure task logger directory (all typed tasks support this) - tasksLoggerConfig := w.getTaskLoggerConfig() - typedLoggerConfig := types.TaskLoggerConfig{ - BaseLogDir: tasksLoggerConfig.BaseLogDir, - MaxTasks: tasksLoggerConfig.MaxTasks, - MaxLogSizeMB: tasksLoggerConfig.MaxLogSizeMB, - EnableConsole: tasksLoggerConfig.EnableConsole, + // Log supported task types for debugging + allFactories := w.registry.GetAll() + glog.Errorf("Available task types: %d", len(allFactories)) + for taskType := range allFactories { + glog.Errorf("Supported task type: %v", taskType) + } + return } - typedTaskInstance.SetLoggerConfig(typedLoggerConfig) - glog.V(2).Infof("Set typed task logger config for %s: %s", task.ID, typedLoggerConfig.BaseLogDir) - // Initialize logging (all typed tasks support this) - taskParams := types.TaskParams{ - VolumeID: task.VolumeID, - Server: task.Server, - Collection: task.Collection, - WorkingDir: taskWorkingDir, - TypedParams: task.TypedParams, - GrpcDialOption: w.config.GrpcDialOption, + taskInstance, err := taskFactory.Create(task.TypedParams) + if err != nil { + w.completeTask(task.ID, false, fmt.Sprintf("failed to create task for %s: %v", task.Type, err)) + glog.Errorf("Worker %s failed to create task %s type %v: %v", w.id, task.ID, task.Type, err) + return } - if err := typedTaskInstance.InitializeTaskLogger(task.ID, w.id, taskParams); err != nil { - glog.Warningf("Failed to initialize task logger for %s: %v", task.ID, err) - } + // Task execution uses the new unified Task interface + glog.V(2).Infof("Executing task %s in working directory: %s", task.ID, taskWorkingDir) // Set progress callback that reports to admin server - typedTaskInstance.SetProgressCallback(func(progress float64) { + taskInstance.SetProgressCallback(func(progress float64) { // Report progress updates to admin server glog.V(2).Infof("Task %s progress: %.1f%%", task.ID, progress) if err := w.adminClient.UpdateTaskProgress(task.ID, progress); err != nil { @@ -448,18 +440,19 @@ func (w *Worker) executeTask(task *types.Task) { } }) - // Execute typed task - err = typedTaskInstance.ExecuteTyped(task.TypedParams) + // Execute task with context + ctx := context.Background() + err = taskInstance.Execute(ctx, task.TypedParams) // Report completion if err != nil { w.completeTask(task.ID, false, err.Error()) w.tasksFailed++ - glog.Errorf("Worker %s failed to execute typed task %s: %v", w.id, task.ID, err) + glog.Errorf("Worker %s failed to execute task %s: %v", w.id, task.ID, err) } else { w.completeTask(task.ID, true, "") w.tasksCompleted++ - glog.Infof("Worker %s completed typed task %s successfully", w.id, task.ID) + glog.Infof("Worker %s completed task %s successfully", w.id, task.ID) } } @@ -558,11 +551,11 @@ func (w *Worker) GetTaskRegistry() *tasks.TaskRegistry { } // GetCurrentTasks returns the current tasks -func (w *Worker) GetCurrentTasks() map[string]*types.Task { +func (w *Worker) GetCurrentTasks() map[string]*types.TaskInput { w.mutex.RLock() defer w.mutex.RUnlock() - tasks := make(map[string]*types.Task) + tasks := make(map[string]*types.TaskInput) for id, task := range w.currentTasks { tasks[id] = task } @@ -571,7 +564,7 @@ func (w *Worker) GetCurrentTasks() map[string]*types.Task { // registerWorker registers the worker with the admin server func (w *Worker) registerWorker() { - workerInfo := &types.Worker{ + workerInfo := &types.WorkerData{ ID: w.id, Capabilities: w.config.Capabilities, MaxConcurrent: w.config.MaxConcurrent, @@ -698,7 +691,7 @@ func (w *Worker) processAdminMessage(message *worker_pb.AdminMessage) { w.id, taskAssign.TaskId, taskAssign.TaskType, taskAssign.Params.VolumeId) // Convert to task and handle it - task := &types.Task{ + task := &types.TaskInput{ ID: taskAssign.TaskId, Type: types.TaskType(taskAssign.TaskType), Status: types.TaskStatusAssigned,