aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorChris Lu <chrislusf@users.noreply.github.com>2025-08-01 11:18:32 -0700
committerGitHub <noreply@github.com>2025-08-01 11:18:32 -0700
commit0975968e71b05368d5f28f788cf863c2042c2696 (patch)
tree5162a5fe3d9c88fb43f49f57326a4fd5b8cde74c
parent1cba609bfa2306cc2885df212febd5ff954aa693 (diff)
downloadseaweedfs-0975968e71b05368d5f28f788cf863c2042c2696.tar.xz
seaweedfs-0975968e71b05368d5f28f788cf863c2042c2696.zip
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
-rw-r--r--docker/admin_integration/docker-compose-ec-test.yml2
-rw-r--r--weed/admin/maintenance/maintenance_integration.go467
-rw-r--r--weed/admin/maintenance/maintenance_scanner.go18
-rw-r--r--weed/admin/maintenance/maintenance_worker.go17
-rw-r--r--weed/admin/topology/active_topology.go301
-rw-r--r--weed/admin/topology/active_topology_test.go181
-rw-r--r--weed/pb/worker.proto1
-rw-r--r--weed/pb/worker_pb/worker.pb.go14
-rw-r--r--weed/worker/client.go22
-rw-r--r--weed/worker/registry.go28
-rw-r--r--weed/worker/tasks/balance/balance.go141
-rw-r--r--weed/worker/tasks/balance/balance_task.go248
-rw-r--r--weed/worker/tasks/balance/detection.go154
-rw-r--r--weed/worker/tasks/balance/execution.go (renamed from weed/worker/tasks/balance/balance_typed.go)0
-rw-r--r--weed/worker/tasks/balance/monitoring.go138
-rw-r--r--weed/worker/tasks/balance/register.go (renamed from weed/worker/tasks/balance/balance_register.go)17
-rw-r--r--weed/worker/tasks/balance/scheduling.go37
-rw-r--r--weed/worker/tasks/base/generic_components.go6
-rw-r--r--weed/worker/tasks/base/registration.go22
-rw-r--r--weed/worker/tasks/base/task_definition.go4
-rw-r--r--weed/worker/tasks/erasure_coding/detection.go314
-rw-r--r--weed/worker/tasks/erasure_coding/ec.go785
-rw-r--r--weed/worker/tasks/erasure_coding/ec_task.go660
-rw-r--r--weed/worker/tasks/erasure_coding/monitoring.go229
-rw-r--r--weed/worker/tasks/erasure_coding/register.go (renamed from weed/worker/tasks/erasure_coding/ec_register.go)17
-rw-r--r--weed/worker/tasks/erasure_coding/scheduling.go40
-rw-r--r--weed/worker/tasks/registry.go64
-rw-r--r--weed/worker/tasks/task.go53
-rw-r--r--weed/worker/tasks/vacuum/detection.go65
-rw-r--r--weed/worker/tasks/vacuum/monitoring.go151
-rw-r--r--weed/worker/tasks/vacuum/register.go (renamed from weed/worker/tasks/vacuum/vacuum_register.go)17
-rw-r--r--weed/worker/tasks/vacuum/scheduling.go37
-rw-r--r--weed/worker/tasks/vacuum/vacuum.go214
-rw-r--r--weed/worker/tasks/vacuum/vacuum_task.go236
-rw-r--r--weed/worker/types/base/task.go99
-rw-r--r--weed/worker/types/data_types.go11
-rw-r--r--weed/worker/types/factory.go52
-rw-r--r--weed/worker/types/task.go189
-rw-r--r--weed/worker/types/task_scheduler.go4
-rw-r--r--weed/worker/types/task_types.go4
-rw-r--r--weed/worker/types/worker.go76
-rw-r--r--weed/worker/types/worker_types.go53
-rw-r--r--weed/worker/worker.go89
43 files changed, 2901 insertions, 2376 deletions
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
index 91cd912f0..91cd912f0 100644
--- a/weed/worker/tasks/balance/balance_typed.go
+++ b/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
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
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
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,