Browse Source

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
pull/7067/head
Chris Lu 2 months ago
committed by GitHub
parent
commit
0975968e71
No known key found for this signature in database GPG Key ID: B5690EEEBB952194
  1. 2
      docker/admin_integration/docker-compose-ec-test.yml
  2. 467
      weed/admin/maintenance/maintenance_integration.go
  3. 18
      weed/admin/maintenance/maintenance_scanner.go
  4. 17
      weed/admin/maintenance/maintenance_worker.go
  5. 301
      weed/admin/topology/active_topology.go
  6. 181
      weed/admin/topology/active_topology_test.go
  7. 1
      weed/pb/worker.proto
  8. 14
      weed/pb/worker_pb/worker.pb.go
  9. 22
      weed/worker/client.go
  10. 28
      weed/worker/registry.go
  11. 141
      weed/worker/tasks/balance/balance.go
  12. 248
      weed/worker/tasks/balance/balance_task.go
  13. 154
      weed/worker/tasks/balance/detection.go
  14. 0
      weed/worker/tasks/balance/execution.go
  15. 138
      weed/worker/tasks/balance/monitoring.go
  16. 17
      weed/worker/tasks/balance/register.go
  17. 37
      weed/worker/tasks/balance/scheduling.go
  18. 6
      weed/worker/tasks/base/generic_components.go
  19. 22
      weed/worker/tasks/base/registration.go
  20. 4
      weed/worker/tasks/base/task_definition.go
  21. 314
      weed/worker/tasks/erasure_coding/detection.go
  22. 785
      weed/worker/tasks/erasure_coding/ec.go
  23. 660
      weed/worker/tasks/erasure_coding/ec_task.go
  24. 229
      weed/worker/tasks/erasure_coding/monitoring.go
  25. 17
      weed/worker/tasks/erasure_coding/register.go
  26. 40
      weed/worker/tasks/erasure_coding/scheduling.go
  27. 64
      weed/worker/tasks/registry.go
  28. 53
      weed/worker/tasks/task.go
  29. 65
      weed/worker/tasks/vacuum/detection.go
  30. 151
      weed/worker/tasks/vacuum/monitoring.go
  31. 17
      weed/worker/tasks/vacuum/register.go
  32. 37
      weed/worker/tasks/vacuum/scheduling.go
  33. 214
      weed/worker/tasks/vacuum/vacuum.go
  34. 236
      weed/worker/tasks/vacuum/vacuum_task.go
  35. 99
      weed/worker/types/base/task.go
  36. 11
      weed/worker/types/data_types.go
  37. 52
      weed/worker/types/factory.go
  38. 189
      weed/worker/types/task.go
  39. 4
      weed/worker/types/task_scheduler.go
  40. 4
      weed/worker/types/task_types.go
  41. 76
      weed/worker/types/worker.go
  42. 53
      weed/worker/types/worker_types.go
  43. 89
      weed/worker/worker.go

2
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

467
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
}

18
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

17
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)
}

301
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

181
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)
})
}

1
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 {

14
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" +

22
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)

28
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)

141
weed/worker/tasks/balance/balance.go

@ -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
}

248
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
})
}

154
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
}

0
weed/worker/tasks/balance/balance_typed.go → weed/worker/tasks/balance/execution.go

138
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
}

17
weed/worker/tasks/balance/balance_register.go → 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,

37
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
}

6
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
}

22
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
}

4
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
}

314
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
}

785
weed/worker/tasks/erasure_coding/ec.go

@ -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")
}

660
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
}

229
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
}

17
weed/worker/tasks/erasure_coding/ec_register.go → 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,

40
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
}

64
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
}

53
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

65
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
}

151
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
}

17
weed/worker/tasks/vacuum/vacuum_register.go → 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,

37
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
}

214
weed/worker/tasks/vacuum/vacuum.go

@ -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()
}

236
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
})
}

99
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
}

11
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)

52
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]

189
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
}

4
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

4
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"`

76
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),
}
}

53
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
}

89
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,

Loading…
Cancel
Save