Browse Source
admin: Refactor task destination planning (#7063)
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 taskspull/7067/head
committed by
GitHub
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
43 changed files with 2901 additions and 2376 deletions
-
2docker/admin_integration/docker-compose-ec-test.yml
-
467weed/admin/maintenance/maintenance_integration.go
-
18weed/admin/maintenance/maintenance_scanner.go
-
17weed/admin/maintenance/maintenance_worker.go
-
301weed/admin/topology/active_topology.go
-
181weed/admin/topology/active_topology_test.go
-
1weed/pb/worker.proto
-
14weed/pb/worker_pb/worker.pb.go
-
22weed/worker/client.go
-
28weed/worker/registry.go
-
141weed/worker/tasks/balance/balance.go
-
248weed/worker/tasks/balance/balance_task.go
-
154weed/worker/tasks/balance/detection.go
-
0weed/worker/tasks/balance/execution.go
-
138weed/worker/tasks/balance/monitoring.go
-
17weed/worker/tasks/balance/register.go
-
37weed/worker/tasks/balance/scheduling.go
-
6weed/worker/tasks/base/generic_components.go
-
22weed/worker/tasks/base/registration.go
-
4weed/worker/tasks/base/task_definition.go
-
314weed/worker/tasks/erasure_coding/detection.go
-
785weed/worker/tasks/erasure_coding/ec.go
-
660weed/worker/tasks/erasure_coding/ec_task.go
-
229weed/worker/tasks/erasure_coding/monitoring.go
-
17weed/worker/tasks/erasure_coding/register.go
-
40weed/worker/tasks/erasure_coding/scheduling.go
-
64weed/worker/tasks/registry.go
-
53weed/worker/tasks/task.go
-
65weed/worker/tasks/vacuum/detection.go
-
151weed/worker/tasks/vacuum/monitoring.go
-
17weed/worker/tasks/vacuum/register.go
-
37weed/worker/tasks/vacuum/scheduling.go
-
214weed/worker/tasks/vacuum/vacuum.go
-
236weed/worker/tasks/vacuum/vacuum_task.go
-
99weed/worker/types/base/task.go
-
11weed/worker/types/data_types.go
-
52weed/worker/types/factory.go
-
189weed/worker/types/task.go
-
4weed/worker/types/task_scheduler.go
-
4weed/worker/types/task_types.go
-
76weed/worker/types/worker.go
-
53weed/worker/types/worker_types.go
-
89weed/worker/worker.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 |
|
||||
} |
|
@ -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 |
||||
|
}) |
||||
|
} |
@ -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 |
||||
|
} |
@ -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 |
||||
|
} |
@ -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") |
|
||||
} |
|
@ -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 |
||||
|
} |
@ -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 |
||||
|
} |
@ -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 |
||||
|
} |
@ -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 |
||||
|
} |
@ -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 |
||||
|
} |
@ -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() |
|
||||
} |
|
@ -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 |
||||
|
}) |
||||
|
} |
@ -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 |
||||
|
} |
@ -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] |
@ -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 |
||||
|
} |
@ -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), |
||||
|
} |
||||
|
} |
Write
Preview
Loading…
Cancel
Save
Reference in new issue