fix(worker): pass compaction revision and file sizes in EC volume copy (#8835)
* fix(worker): pass compaction revision and file sizes in EC volume copy The worker EC task was sending CopyFile requests without the current compaction revision (defaulting to 0) and with StopOffset set to math.MaxInt64. After a vacuum compaction this caused the volume server to reject the copy or return stale data. Read the volume file status first and forward the compaction revision and actual file sizes so the copy is consistent with the compacted volume. * propagate erasure coding task context * fix(worker): validate volume file status and detect short copies Reject zero dat file size from ReadVolumeFileStatus — a zero-sized snapshot would produce 0-byte copies and broken EC shards. After streaming, verify totalBytes matches the expected stopOffset and return an error on short copies instead of logging success. * fix(worker): reject zero idx file size in volume status validation A non-empty dat with zero idx indicates an empty or corrupt volume. Without this guard, copyFileFromSource gets stopOffset=0, produces a 0-byte .idx, passes the short-copy check, and generateEcShardsLocally runs against a volume with no index. * fix fake plugin volume file status * fix plugin volume balance test fixtures
This commit is contained in:
@@ -4,7 +4,6 @@ import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"math"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"strings"
|
||||
@@ -146,14 +145,14 @@ func (t *ErasureCodingTask) Execute(ctx context.Context, params *worker_pb.TaskP
|
||||
// Step 1: Mark volume readonly
|
||||
t.ReportProgressWithStage(10.0, "Marking volume readonly")
|
||||
t.GetLogger().Info("Marking volume readonly")
|
||||
if err := t.markVolumeReadonly(); err != nil {
|
||||
if err := t.markVolumeReadonly(ctx); err != nil {
|
||||
return fmt.Errorf("failed to mark volume readonly: %v", err)
|
||||
}
|
||||
|
||||
// Step 2: Copy volume files to worker
|
||||
t.ReportProgressWithStage(25.0, "Copying volume files to worker")
|
||||
t.GetLogger().Info("Copying volume files to worker")
|
||||
localFiles, err := t.copyVolumeFilesToWorker(taskWorkDir)
|
||||
localFiles, err := t.copyVolumeFilesToWorker(ctx, taskWorkDir)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to copy volume files: %v", err)
|
||||
}
|
||||
@@ -183,7 +182,7 @@ func (t *ErasureCodingTask) Execute(ctx context.Context, params *worker_pb.TaskP
|
||||
// Step 6: Delete original volume
|
||||
t.ReportProgressWithStage(90.0, "Deleting original volume")
|
||||
t.GetLogger().Info("Deleting original volume")
|
||||
if err := t.deleteOriginalVolume(); err != nil {
|
||||
if err := t.deleteOriginalVolume(ctx); err != nil {
|
||||
return fmt.Errorf("failed to delete original volume: %v", err)
|
||||
}
|
||||
|
||||
@@ -250,10 +249,10 @@ func (t *ErasureCodingTask) GetProgress() float64 {
|
||||
// Helper methods for actual EC operations
|
||||
|
||||
// markVolumeReadonly marks the volume as readonly on the source server
|
||||
func (t *ErasureCodingTask) markVolumeReadonly() error {
|
||||
func (t *ErasureCodingTask) markVolumeReadonly(ctx context.Context) error {
|
||||
return operation.WithVolumeServerClient(false, pb.ServerAddress(t.server), t.grpcDialOption,
|
||||
func(client volume_server_pb.VolumeServerClient) error {
|
||||
_, err := client.VolumeMarkReadonly(context.Background(), &volume_server_pb.VolumeMarkReadonlyRequest{
|
||||
_, err := client.VolumeMarkReadonly(ctx, &volume_server_pb.VolumeMarkReadonlyRequest{
|
||||
VolumeId: t.volumeID,
|
||||
})
|
||||
return err
|
||||
@@ -261,18 +260,26 @@ func (t *ErasureCodingTask) markVolumeReadonly() error {
|
||||
}
|
||||
|
||||
// copyVolumeFilesToWorker copies .dat and .idx files from source server to local worker
|
||||
func (t *ErasureCodingTask) copyVolumeFilesToWorker(workDir string) (map[string]string, error) {
|
||||
func (t *ErasureCodingTask) copyVolumeFilesToWorker(ctx context.Context, workDir string) (map[string]string, error) {
|
||||
localFiles := make(map[string]string)
|
||||
|
||||
fileStatus, err := t.readSourceVolumeFileStatus(ctx)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to read source volume file status: %v", err)
|
||||
}
|
||||
|
||||
t.GetLogger().WithFields(map[string]interface{}{
|
||||
"volume_id": t.volumeID,
|
||||
"source": t.server,
|
||||
"working_dir": workDir,
|
||||
"volume_id": t.volumeID,
|
||||
"source": t.server,
|
||||
"working_dir": workDir,
|
||||
"compaction_revision": fileStatus.GetCompactionRevision(),
|
||||
"dat_file_size_bytes": fileStatus.GetDatFileSize(),
|
||||
"idx_file_size_bytes": fileStatus.GetIdxFileSize(),
|
||||
}).Info("Starting volume file copy from source server")
|
||||
|
||||
// Copy .dat file
|
||||
datFile := filepath.Join(workDir, fmt.Sprintf("%d.dat", t.volumeID))
|
||||
if err := t.copyFileFromSource(".dat", datFile); err != nil {
|
||||
if err := t.copyFileFromSource(ctx, ".dat", datFile, fileStatus.GetCompactionRevision(), fileStatus.GetDatFileSize()); err != nil {
|
||||
return nil, fmt.Errorf("failed to copy .dat file: %v", err)
|
||||
}
|
||||
localFiles["dat"] = datFile
|
||||
@@ -289,7 +296,7 @@ func (t *ErasureCodingTask) copyVolumeFilesToWorker(workDir string) (map[string]
|
||||
|
||||
// Copy .idx file
|
||||
idxFile := filepath.Join(workDir, fmt.Sprintf("%d.idx", t.volumeID))
|
||||
if err := t.copyFileFromSource(".idx", idxFile); err != nil {
|
||||
if err := t.copyFileFromSource(ctx, ".idx", idxFile, fileStatus.GetCompactionRevision(), fileStatus.GetIdxFileSize()); err != nil {
|
||||
return nil, fmt.Errorf("failed to copy .idx file: %v", err)
|
||||
}
|
||||
localFiles["idx"] = idxFile
|
||||
@@ -307,15 +314,38 @@ func (t *ErasureCodingTask) copyVolumeFilesToWorker(workDir string) (map[string]
|
||||
return localFiles, nil
|
||||
}
|
||||
|
||||
func (t *ErasureCodingTask) readSourceVolumeFileStatus(ctx context.Context) (*volume_server_pb.ReadVolumeFileStatusResponse, error) {
|
||||
var statusResp *volume_server_pb.ReadVolumeFileStatusResponse
|
||||
err := operation.WithVolumeServerClient(false, pb.ServerAddress(t.server), t.grpcDialOption,
|
||||
func(client volume_server_pb.VolumeServerClient) error {
|
||||
var readErr error
|
||||
statusResp, readErr = client.ReadVolumeFileStatus(ctx, &volume_server_pb.ReadVolumeFileStatusRequest{
|
||||
VolumeId: t.volumeID,
|
||||
})
|
||||
return readErr
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if statusResp.GetDatFileSize() == 0 {
|
||||
return nil, fmt.Errorf("volume %d on %s reports zero dat file size", t.volumeID, t.server)
|
||||
}
|
||||
if statusResp.GetIdxFileSize() == 0 {
|
||||
return nil, fmt.Errorf("volume %d on %s reports zero idx file size with non-empty dat", t.volumeID, t.server)
|
||||
}
|
||||
return statusResp, nil
|
||||
}
|
||||
|
||||
// copyFileFromSource copies a file from source server to local path using gRPC streaming
|
||||
func (t *ErasureCodingTask) copyFileFromSource(ext, localPath string) error {
|
||||
func (t *ErasureCodingTask) copyFileFromSource(ctx context.Context, ext, localPath string, compactionRevision uint32, stopOffset uint64) error {
|
||||
return operation.WithVolumeServerClient(false, pb.ServerAddress(t.server), t.grpcDialOption,
|
||||
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),
|
||||
stream, err := client.CopyFile(ctx, &volume_server_pb.CopyFileRequest{
|
||||
VolumeId: t.volumeID,
|
||||
Collection: t.collection,
|
||||
Ext: ext,
|
||||
CompactionRevision: compactionRevision,
|
||||
StopOffset: stopOffset,
|
||||
})
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to initiate file copy: %v", err)
|
||||
@@ -348,6 +378,9 @@ func (t *ErasureCodingTask) copyFileFromSource(ext, localPath string) error {
|
||||
}
|
||||
}
|
||||
|
||||
if totalBytes != int64(stopOffset) {
|
||||
return fmt.Errorf("short copy of %s: got %d bytes, expected %d", ext, totalBytes, stopOffset)
|
||||
}
|
||||
glog.V(1).Infof("Successfully copied %s (%d bytes) from %s to %s", ext, totalBytes, t.server, localPath)
|
||||
return nil
|
||||
})
|
||||
@@ -468,7 +501,7 @@ func (t *ErasureCodingTask) mountEcShards() error {
|
||||
}
|
||||
|
||||
// deleteOriginalVolume deletes the original volume and all its replicas from all servers
|
||||
func (t *ErasureCodingTask) deleteOriginalVolume() error {
|
||||
func (t *ErasureCodingTask) deleteOriginalVolume(ctx context.Context) error {
|
||||
// Get replicas from task parameters (set during detection)
|
||||
replicas := t.getReplicas()
|
||||
|
||||
@@ -497,7 +530,7 @@ func (t *ErasureCodingTask) deleteOriginalVolume() error {
|
||||
|
||||
err := operation.WithVolumeServerClient(false, pb.ServerAddress(replicaServer), t.grpcDialOption,
|
||||
func(client volume_server_pb.VolumeServerClient) error {
|
||||
_, err := client.VolumeDelete(context.Background(), &volume_server_pb.VolumeDeleteRequest{
|
||||
_, err := client.VolumeDelete(ctx, &volume_server_pb.VolumeDeleteRequest{
|
||||
VolumeId: t.volumeID,
|
||||
OnlyEmpty: false, // Force delete since we've created EC shards
|
||||
})
|
||||
|
||||
108
weed/worker/tasks/erasure_coding/ec_task_test.go
Normal file
108
weed/worker/tasks/erasure_coding/ec_task_test.go
Normal file
@@ -0,0 +1,108 @@
|
||||
package erasure_coding
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
"net/http"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/seaweedfs/seaweedfs/test/volume_server/framework"
|
||||
"github.com/seaweedfs/seaweedfs/test/volume_server/matrix"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb"
|
||||
"github.com/stretchr/testify/require"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
)
|
||||
|
||||
func TestCopyVolumeFilesToWorkerUsesCurrentCompactionRevision(t *testing.T) {
|
||||
if testing.Short() {
|
||||
t.Skip("skipping integration test in short mode")
|
||||
}
|
||||
|
||||
clusterHarness := framework.StartVolumeCluster(t, matrix.P1())
|
||||
conn, grpcClient := framework.DialVolumeServer(t, clusterHarness.VolumeGRPCAddress())
|
||||
defer conn.Close()
|
||||
|
||||
const volumeID = uint32(951)
|
||||
framework.AllocateVolume(t, grpcClient, volumeID, "")
|
||||
|
||||
httpClient := framework.NewHTTPClient()
|
||||
|
||||
liveFID := framework.NewFileID(volumeID, 1001, 0x1111AAAA)
|
||||
liveUploadResp := framework.UploadBytes(t, httpClient, clusterHarness.VolumeAdminURL(), liveFID, []byte("live-payload-for-ec-copy"))
|
||||
_ = framework.ReadAllAndClose(t, liveUploadResp)
|
||||
require.Equal(t, http.StatusCreated, liveUploadResp.StatusCode)
|
||||
|
||||
deletedFID := framework.NewFileID(volumeID, 1002, 0x2222BBBB)
|
||||
deletedUploadResp := framework.UploadBytes(t, httpClient, clusterHarness.VolumeAdminURL(), deletedFID, []byte("deleted-payload-for-vacuum"))
|
||||
_ = framework.ReadAllAndClose(t, deletedUploadResp)
|
||||
require.Equal(t, http.StatusCreated, deletedUploadResp.StatusCode)
|
||||
|
||||
deleteReq, err := http.NewRequest(http.MethodDelete, clusterHarness.VolumeAdminURL()+"/"+deletedFID, nil)
|
||||
require.NoError(t, err)
|
||||
deleteResp := framework.DoRequest(t, httpClient, deleteReq)
|
||||
_ = framework.ReadAllAndClose(t, deleteResp)
|
||||
require.Equal(t, http.StatusAccepted, deleteResp.StatusCode)
|
||||
|
||||
compactVolumeOnce(t, grpcClient, volumeID)
|
||||
|
||||
task := NewErasureCodingTask(
|
||||
"copy-after-compaction",
|
||||
clusterHarness.VolumeServerAddress(),
|
||||
volumeID,
|
||||
"",
|
||||
grpc.WithTransportCredentials(insecure.NewCredentials()),
|
||||
)
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
|
||||
defer cancel()
|
||||
|
||||
require.NoError(t, task.markVolumeReadonly(ctx))
|
||||
|
||||
fileStatus, err := task.readSourceVolumeFileStatus(ctx)
|
||||
require.NoError(t, err)
|
||||
require.Greater(t, fileStatus.GetCompactionRevision(), uint32(0))
|
||||
|
||||
localFiles, err := task.copyVolumeFilesToWorker(ctx, t.TempDir())
|
||||
require.NoError(t, err)
|
||||
|
||||
datInfo, err := os.Stat(localFiles["dat"])
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, int64(fileStatus.GetDatFileSize()), datInfo.Size())
|
||||
|
||||
idxInfo, err := os.Stat(localFiles["idx"])
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, int64(fileStatus.GetIdxFileSize()), idxInfo.Size())
|
||||
}
|
||||
|
||||
func compactVolumeOnce(t *testing.T, grpcClient volume_server_pb.VolumeServerClient, volumeID uint32) {
|
||||
t.Helper()
|
||||
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
|
||||
defer cancel()
|
||||
|
||||
compactStream, err := grpcClient.VacuumVolumeCompact(ctx, &volume_server_pb.VacuumVolumeCompactRequest{
|
||||
VolumeId: volumeID,
|
||||
})
|
||||
require.NoError(t, err)
|
||||
|
||||
for {
|
||||
_, err = compactStream.Recv()
|
||||
if err == io.EOF {
|
||||
break
|
||||
}
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
_, err = grpcClient.VacuumVolumeCommit(ctx, &volume_server_pb.VacuumVolumeCommitRequest{
|
||||
VolumeId: volumeID,
|
||||
})
|
||||
require.NoError(t, err)
|
||||
|
||||
_, err = grpcClient.VacuumVolumeCleanup(ctx, &volume_server_pb.VacuumVolumeCleanupRequest{
|
||||
VolumeId: volumeID,
|
||||
})
|
||||
require.NoError(t, err)
|
||||
}
|
||||
Reference in New Issue
Block a user