Batch volume balance: run multiple moves per job (#8561)
* proto: add BalanceMoveSpec and batch fields to BalanceTaskParams Add BalanceMoveSpec message for encoding individual volume moves, and max_concurrent_moves + repeated moves fields to BalanceTaskParams to support batching multiple volume moves in a single job. * balance handler: add batch execution with concurrent volume moves Refactor Execute() into executeSingleMove() (backward compatible) and executeBatchMoves() which runs multiple volume moves concurrently using a semaphore-bounded goroutine pool. When BalanceTaskParams.Moves is populated, the batch path is taken; otherwise the single-move path. Includes aggregate progress reporting across concurrent moves, per-move error collection, and partial failure support. * balance handler: add batch config fields to Descriptor and worker config Add max_concurrent_moves and batch_size fields to the worker config form and deriveBalanceWorkerConfig(). These control how many volume moves run concurrently within a batch job and the maximum batch size. * balance handler: group detection proposals into batch jobs When batch_size > 1, the Detect method groups detection results into batch proposals where each proposal encodes multiple BalanceMoveSpec entries in BalanceTaskParams.Moves. Single-result batches fall back to the existing single-move proposal format for backward compatibility. * admin UI: add volume balance execution plan and batch badge Add renderBalanceExecutionPlan() for rich rendering of volume balance jobs in the job detail modal. Single-move jobs show source/target/volume info; batch jobs show a moves table with all volume moves. Add batch badge (e.g., "5 moves") next to job type in the execution jobs table when the job has batch=true label. * Update plugin_templ.go * fix: detection algorithm uses greedy target instead of divergent topology scores The detection loop tracked effective volume counts via an adjustments map, but createBalanceTask independently called planBalanceDestination which used the topology's LoadCount — a separate, unadjusted source of truth. This divergence caused multiple moves to pile onto the same server. Changes: - Add resolveBalanceDestination to resolve the detection loop's greedy target (minServer) rather than independently picking a destination - Add oscillation guard: stop when max-min <= 1 since no single move can improve the balance beyond that point - Track unseeded destinations: if a target server wasn't in the initial serverVolumeCounts, add it so subsequent iterations include it - Add TestDetection_UnseededDestinationDoesNotOverload * fix: handler force_move propagation, partial failure, deterministic dedupe - Propagate ForceMove from outer BalanceTaskParams to individual move TaskParams so batch moves respect the force_move flag - Fix partial failure: mark job successful if at least one move succeeded (succeeded > 0 || failed == 0) to avoid re-running already-completed moves on retry - Use SHA-256 hash for deterministic dedupe key fallback instead of time.Now().UnixNano() which is non-deterministic - Remove unused successDetails variable - Extract maxProposalStringLength constant to replace magic number 200 * admin UI: use template literals in balance execution plan rendering * fix: integration test handles batch proposals from batched detection With batch_size=20, all moves are grouped into a single proposal containing BalanceParams.Moves instead of top-level Sources/Targets. Update assertions to handle both batch and single-move proposal formats. * fix: verify volume size on target before deleting source during balance Add a pre-delete safety check that reads the volume file status on both source and target, then compares .dat file size and file count. If they don't match, the move is aborted — leaving the source intact rather than risking irreversible data loss. Also removes the redundant mountVolume call since VolumeCopy already mounts the volume on the target server. * fix: clamp maxConcurrent, serialize progress sends, validate config as int64 - Clamp maxConcurrentMoves to defaultMaxConcurrentMoves before creating the semaphore so a stale or malicious job cannot request unbounded concurrent volume moves - Extend progressMu to cover sender.SendProgress calls since the underlying gRPC stream is not safe for concurrent writes - Perform bounds checks on max_concurrent_moves and batch_size in int64 space before casting to int, avoiding potential overflow on 32-bit * fix: check disk capacity in resolveBalanceDestination Skip disks where VolumeCount >= MaxVolumeCount so the detection loop does not propose moves to a full disk that would fail at execution time. * test: rename unseeded destination test to match actual behavior The test exercises a server with 0 volumes that IS seeded from topology (matching disk type), not an unseeded destination. Rename to TestDetection_ZeroVolumeServerIncludedInBalance and fix comments. * test: tighten integration test to assert exactly one batch proposal With default batch_size=20, all moves should be grouped into a single batch proposal. Assert len(proposals)==1 and require BalanceParams with Moves, removing the legacy single-move else branch. * fix: propagate ctx to RPCs and restore source writability on abort - All helper methods (markVolumeReadonly, copyVolume, tailVolume, readVolumeFileStatus, deleteVolume) now accept a context parameter instead of using context.Background(), so Execute's ctx propagates cancellation and timeouts into every volume server RPC - Add deferred cleanup that restores the source volume to writable if any step after markVolumeReadonly fails, preventing the source from being left permanently readonly on abort - Add markVolumeWritable helper using VolumeMarkWritableRequest * fix: deep-copy protobuf messages in test recording sender Use proto.Clone in recordingExecutionSender to store immutable snapshots of JobProgressUpdate and JobCompleted, preventing assertions from observing mutations if the handler reuses message pointers. * fix: add VolumeMarkWritable and ReadVolumeFileStatus to fake volume server The balance task now calls ReadVolumeFileStatus for pre-delete verification and VolumeMarkWritable to restore writability on abort. Add both RPCs to the test fake, and drop the mountCalls assertion since BalanceTask no longer calls VolumeMount directly (VolumeCopy handles it). * fix: use maxConcurrentMovesLimit (50) for clamp, not defaultMaxConcurrentMoves defaultMaxConcurrentMoves (5) is the fallback when the field is unset, not an upper bound. Clamping to it silently overrides valid config values like 10/20/50. Introduce maxConcurrentMovesLimit (50) matching the descriptor's MaxValue and clamp to that instead. * fix: cancel batch moves on progress stream failure Derive a cancellable batchCtx from the caller's ctx. If sender.SendProgress returns an error (client disconnect, context cancelled), capture it, skip further sends, and cancel batchCtx so in-flight moves abort via their propagated context rather than running blind to completion. * fix: bound cleanup timeout and validate batch move fields - Use a 30-second timeout for the deferred markVolumeWritable cleanup instead of context.Background() which can block indefinitely if the volume server is unreachable - Validate required fields (VolumeID, SourceNode, TargetNode) before appending moves to a batch proposal, skipping invalid entries - Fall back to a single-move proposal when filtering leaves only one valid move in a batch * fix: cancel task execution on SendProgress stream failure All handler progress callbacks previously ignored SendProgress errors, allowing tasks to continue executing after the client disconnected. Now each handler creates a derived cancellable context and cancels it on the first SendProgress error, stopping the in-flight task promptly. Handlers fixed: erasure_coding, vacuum, volume_balance (single-move), and admin_script (breaks command loop on send failure). * fix: validate batch moves before scheduling in executeBatchMoves Reject empty batches, enforce a hard upper bound (100 moves), and filter out nil or incomplete move specs (missing source/target/volume) before allocating progress tracking and launching goroutines. * test: add batch balance execution integration test Tests the batch move path with 3 volumes, max concurrency 2, using fake volume servers. Verifies all moves complete with correct readonly, copy, tail, and delete RPC counts. * test: add MarkWritableCount and ReadFileStatusCount accessors Expose the markWritableCalls and readFileStatusCalls counters on the fake volume server, following the existing MarkReadonlyCount pattern. * fix: oscillation guard uses global effective counts for heterogeneous capacity The oscillation guard (max-min <= 1) previously used maxServer/minServer which are determined by utilization ratio. With heterogeneous capacity, maxServer by utilization can have fewer raw volumes than minServer, producing a negative diff and incorrectly triggering the guard. Now scans all servers' effective counts to find the true global max/min volume counts, so the guard works correctly regardless of whether utilization-based or raw-count balancing is used. * fix: admin script handler breaks outer loop on SendProgress failure The break on SendProgress error inside the shell.Commands scan only exited the inner loop, letting the outer command loop continue executing commands on a broken stream. Use a sendBroken flag to propagate the break to the outer execCommands loop.
This commit is contained in:
@@ -37,19 +37,25 @@ func TestVolumeBalanceDetectionIntegration(t *testing.T) {
|
||||
MasterGrpcAddresses: []string{master.Address()},
|
||||
}, 10)
|
||||
require.NoError(t, err)
|
||||
// With 10 volumes on one server and 1 on the other (avg=5.5),
|
||||
// multiple balance moves should be detected until imbalance is within threshold.
|
||||
require.Greater(t, len(proposals), 1, "expected multiple balance proposals")
|
||||
// With default batch_size=20 and 10 overloaded volumes vs 1 underloaded,
|
||||
// all moves are grouped into a single batch proposal.
|
||||
require.Len(t, proposals, 1, "expected exactly one batch proposal")
|
||||
|
||||
for _, proposal := range proposals {
|
||||
require.Equal(t, "volume_balance", proposal.JobType)
|
||||
paramsValue := proposal.Parameters["task_params_pb"]
|
||||
require.NotNil(t, paramsValue)
|
||||
proposal := proposals[0]
|
||||
require.Equal(t, "volume_balance", proposal.JobType)
|
||||
paramsValue := proposal.Parameters["task_params_pb"]
|
||||
require.NotNil(t, paramsValue)
|
||||
|
||||
params := &worker_pb.TaskParams{}
|
||||
require.NoError(t, proto.Unmarshal(paramsValue.GetBytesValue(), params))
|
||||
require.NotEmpty(t, params.Sources)
|
||||
require.NotEmpty(t, params.Targets)
|
||||
params := &worker_pb.TaskParams{}
|
||||
require.NoError(t, proto.Unmarshal(paramsValue.GetBytesValue(), params))
|
||||
|
||||
bp := params.GetBalanceParams()
|
||||
require.NotNil(t, bp, "expected BalanceParams in batch proposal")
|
||||
require.Greater(t, len(bp.Moves), 1, "batch proposal should contain multiple moves")
|
||||
for _, move := range bp.Moves {
|
||||
require.NotZero(t, move.VolumeId)
|
||||
require.NotEmpty(t, move.SourceNode)
|
||||
require.NotEmpty(t, move.TargetNode)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -8,10 +8,12 @@ import (
|
||||
|
||||
pluginworkers "github.com/seaweedfs/seaweedfs/test/plugin_workers"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/plugin_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/worker_pb"
|
||||
pluginworker "github.com/seaweedfs/seaweedfs/weed/plugin/worker"
|
||||
"github.com/stretchr/testify/require"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
func TestVolumeBalanceExecutionIntegration(t *testing.T) {
|
||||
@@ -60,8 +62,85 @@ func TestVolumeBalanceExecutionIntegration(t *testing.T) {
|
||||
require.GreaterOrEqual(t, source.MarkReadonlyCount(), 1)
|
||||
require.GreaterOrEqual(t, len(source.DeleteRequests()), 1)
|
||||
|
||||
copyCalls, mountCalls, tailCalls := target.BalanceStats()
|
||||
copyCalls, _, tailCalls := target.BalanceStats()
|
||||
require.GreaterOrEqual(t, copyCalls, 1)
|
||||
require.GreaterOrEqual(t, mountCalls, 1)
|
||||
require.GreaterOrEqual(t, tailCalls, 1)
|
||||
}
|
||||
|
||||
func TestVolumeBalanceBatchExecutionIntegration(t *testing.T) {
|
||||
dialOption := grpc.WithTransportCredentials(insecure.NewCredentials())
|
||||
handler := pluginworker.NewVolumeBalanceHandler(dialOption)
|
||||
harness := pluginworkers.NewHarness(t, pluginworkers.HarnessConfig{
|
||||
WorkerOptions: pluginworker.WorkerOptions{
|
||||
GrpcDialOption: dialOption,
|
||||
},
|
||||
Handlers: []pluginworker.JobHandler{handler},
|
||||
})
|
||||
harness.WaitForJobType("volume_balance")
|
||||
|
||||
// Create one source and one target fake volume server.
|
||||
source := pluginworkers.NewVolumeServer(t, "")
|
||||
target := pluginworkers.NewVolumeServer(t, "")
|
||||
|
||||
// Build a batch job with 3 volume moves from source → target.
|
||||
volumeIDs := []uint32{401, 402, 403}
|
||||
moves := make([]*worker_pb.BalanceMoveSpec, len(volumeIDs))
|
||||
for i, vid := range volumeIDs {
|
||||
moves[i] = &worker_pb.BalanceMoveSpec{
|
||||
VolumeId: vid,
|
||||
SourceNode: source.Address(),
|
||||
TargetNode: target.Address(),
|
||||
Collection: "batch-test",
|
||||
}
|
||||
}
|
||||
|
||||
params := &worker_pb.TaskParams{
|
||||
TaskId: "batch-balance-test",
|
||||
TaskParams: &worker_pb.TaskParams_BalanceParams{
|
||||
BalanceParams: &worker_pb.BalanceTaskParams{
|
||||
MaxConcurrentMoves: 2,
|
||||
Moves: moves,
|
||||
},
|
||||
},
|
||||
}
|
||||
paramBytes, err := proto.Marshal(params)
|
||||
require.NoError(t, err)
|
||||
|
||||
job := &plugin_pb.JobSpec{
|
||||
JobId: "batch-balance-test",
|
||||
JobType: "volume_balance",
|
||||
Parameters: map[string]*plugin_pb.ConfigValue{
|
||||
"task_params_pb": {
|
||||
Kind: &plugin_pb.ConfigValue_BytesValue{BytesValue: paramBytes},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
|
||||
defer cancel()
|
||||
|
||||
result, err := harness.Plugin().ExecuteJob(ctx, job, nil, 1)
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, result)
|
||||
require.True(t, result.Success, "batch balance job should succeed; result: %+v", result)
|
||||
|
||||
// Each of the 3 moves should have marked the source readonly and deleted.
|
||||
require.Equal(t, len(volumeIDs), source.MarkReadonlyCount(),
|
||||
"each move should mark source volume readonly")
|
||||
require.Equal(t, len(volumeIDs), len(source.DeleteRequests()),
|
||||
"each move should delete the source volume")
|
||||
|
||||
// Verify delete requests reference the expected volume IDs.
|
||||
deletedVols := make(map[uint32]bool)
|
||||
for _, req := range source.DeleteRequests() {
|
||||
deletedVols[req.VolumeId] = true
|
||||
}
|
||||
for _, vid := range volumeIDs {
|
||||
require.True(t, deletedVols[vid], "volume %d should have been deleted from source", vid)
|
||||
}
|
||||
|
||||
// Target should have received copy and tail calls for all 3 volumes.
|
||||
copyCalls, _, tailCalls := target.BalanceStats()
|
||||
require.Equal(t, len(volumeIDs), copyCalls, "target should receive one copy per volume")
|
||||
require.Equal(t, len(volumeIDs), tailCalls, "target should receive one tail per volume")
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user