shell: add volume.check.disk to fix inconsistency for replicated volumes

fix https://github.com/chrislusf/seaweedfs/issues/1923
This commit is contained in:
Chris Lu 2021-03-22 00:03:16 -07:00
parent df461402cc
commit b465095db1
12 changed files with 1565 additions and 801 deletions

View file

@ -1,6 +1,6 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.24.0
// protoc-gen-go v1.25.0
// protoc v3.12.3
// source: filer.proto

View file

@ -1,6 +1,6 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.24.0
// protoc-gen-go v1.25.0
// protoc v3.12.3
// source: iam.proto

View file

@ -1,6 +1,6 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.24.0
// protoc-gen-go v1.25.0
// protoc v3.12.3
// source: master.proto

View file

@ -1,6 +1,6 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.24.0
// protoc-gen-go v1.25.0
// protoc v3.12.3
// source: messaging.proto

View file

@ -52,6 +52,11 @@ service VolumeServer {
rpc CopyFile (CopyFileRequest) returns (stream CopyFileResponse) {
}
rpc ReadNeedleBlob (ReadNeedleBlobRequest) returns (ReadNeedleBlobResponse) {
}
rpc WriteNeedleBlob (WriteNeedleBlobRequest) returns (WriteNeedleBlobResponse) {
}
rpc VolumeTailSender (VolumeTailSenderRequest) returns (stream VolumeTailSenderResponse) {
}
rpc VolumeTailReceiver (VolumeTailReceiverRequest) returns (VolumeTailReceiverResponse) {
@ -253,6 +258,25 @@ message CopyFileResponse {
bytes file_content = 1;
}
message ReadNeedleBlobRequest {
uint32 volume_id = 1;
uint64 needle_id = 2;
int64 offset = 3; // actual offset
int32 size = 4;
}
message ReadNeedleBlobResponse {
bytes needle_blob = 1;
}
message WriteNeedleBlobRequest {
uint32 volume_id = 1;
uint64 needle_id = 2;
int32 size = 3;
bytes needle_blob = 4;
}
message WriteNeedleBlobResponse {
}
message VolumeTailSenderRequest {
uint32 volume_id = 1;
uint64 since_ns = 2;

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,38 @@
package weed_server
import (
"context"
"fmt"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
"github.com/chrislusf/seaweedfs/weed/storage/types"
)
func (vs *VolumeServer) ReadNeedleBlob(ctx context.Context, req *volume_server_pb.ReadNeedleBlobRequest) (resp *volume_server_pb.ReadNeedleBlobResponse, err error) {
resp = &volume_server_pb.ReadNeedleBlobResponse{}
v := vs.store.GetVolume(needle.VolumeId(req.VolumeId))
if v == nil {
return nil, fmt.Errorf("not found volume id %d", req.VolumeId)
}
resp.NeedleBlob, err = v.ReadNeedleBlob(req.Offset, types.Size(req.Size))
if err != nil {
return nil, fmt.Errorf("read needle blob offset %d size %d: %v", req.Offset, req.Size, err)
}
return resp, nil
}
func (vs *VolumeServer) WriteNeedleBlob(ctx context.Context, req *volume_server_pb.WriteNeedleBlobRequest) (resp *volume_server_pb.WriteNeedleBlobResponse, err error) {
resp = &volume_server_pb.WriteNeedleBlobResponse{}
v := vs.store.GetVolume(needle.VolumeId(req.VolumeId))
if v == nil {
return nil, fmt.Errorf("not found volume id %d", req.VolumeId)
}
if err = v.WriteNeedleBlob(types.NeedleId(req.NeedleId), req.NeedleBlob, types.Size(req.Size)); err != nil {
return nil, fmt.Errorf("write blob needle %d: %v", req.NeedleId, req.Size, err)
}
return resp, nil
}

View file

@ -0,0 +1,243 @@
package shell
import (
"bytes"
"context"
"flag"
"fmt"
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage/needle_map"
"io"
"math"
"sort"
)
func init() {
Commands = append(Commands, &commandVolumeCheckDisk{})
}
type commandVolumeCheckDisk struct {
env *CommandEnv
}
func (c *commandVolumeCheckDisk) Name() string {
return "volume.check.disk"
}
func (c *commandVolumeCheckDisk) Help() string {
return `check all replicated volumes to find and fix inconsistencies
How it works:
find all volumes that are replicated
for each volume id, if there are more than 2 replicas, find one pair with the largest 2 in file count.
for the pair volume A and B
append entries in A and not in B to B
append entries in B and not in A to A
`
}
func (c *commandVolumeCheckDisk) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
if err = commandEnv.confirmIsLocked(); err != nil {
return
}
fsckCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
verbose := fsckCommand.Bool("v", false, "verbose mode")
applyChanges := fsckCommand.Bool("force", false, "apply the fix")
if err = fsckCommand.Parse(args); err != nil {
return nil
}
c.env = commandEnv
// collect topology information
topologyInfo, _, err := collectTopologyInfo(commandEnv)
if err != nil {
return err
}
volumeReplicas, _ := collectVolumeReplicaLocations(topologyInfo)
// pick 1 pairs of volume replica
fileCount := func(replica *VolumeReplica) uint64 {
return replica.info.FileCount - replica.info.DeleteCount
}
aDB, bDB := needle_map.NewMemDb(), needle_map.NewMemDb()
defer aDB.Close()
defer bDB.Close()
for _, replicas := range volumeReplicas {
sort.Slice(replicas, func(i, j int) bool {
return fileCount(replicas[i]) > fileCount(replicas[j])
})
for len(replicas) >= 2 {
a, b := replicas[0], replicas[1]
if fileCount(a) == fileCount(b) {
replicas = replicas[1:]
continue
}
if a.info.ReadOnly || b.info.ReadOnly {
fmt.Fprintf(writer, "skipping readonly volume %d on %s and %s\n", a.info.Id, a.location.dataNode.Id, b.location.dataNode.Id)
continue
}
// reset index db
aDB.Close()
bDB.Close()
aDB, bDB = needle_map.NewMemDb(), needle_map.NewMemDb()
// read index db
if err := c.readIndexDatabase(aDB, a.info.Collection, a.info.Id, a.location.dataNode.Id, *verbose, writer); err != nil {
return err
}
if err := c.readIndexDatabase(bDB, b.info.Collection, b.info.Id, b.location.dataNode.Id, *verbose, writer); err != nil {
return err
}
// find and make up the differnces
if err := c.doVolumeCheckDisk(aDB, bDB, a, b, *verbose, writer, *applyChanges); err != nil {
return err
}
if err := c.doVolumeCheckDisk(bDB, aDB, b, a, *verbose, writer, *applyChanges); err != nil {
return err
}
replicas = replicas[1:]
}
}
return nil
}
func (c *commandVolumeCheckDisk) doVolumeCheckDisk(subtrahend, minuend *needle_map.MemDb, source, target *VolumeReplica, verbose bool, writer io.Writer, applyChanges bool) error {
// find missing keys
// hash join, can be more efficient
var missingNeedles []needle_map.NeedleValue
var counter int
subtrahend.AscendingVisit(func(value needle_map.NeedleValue) error {
counter++
if _, found := minuend.Get(value.Key); !found {
missingNeedles = append(missingNeedles, value)
}
return nil
})
fmt.Fprintf(writer, "%s has %d entries, %s missed %d entries\n", source.location.dataNode.Id, counter, target.location.dataNode.Id, len(missingNeedles))
for _, needleValue := range missingNeedles {
needleBlob, err := c.readSourceNeedleBlob(source.location.dataNode.Id, source.info.Id, needleValue)
if err != nil {
return err
}
if !applyChanges {
continue
}
if verbose {
fmt.Fprintf(writer, "%s read %s \n", source.location.dataNode.Id, counter, target.location.dataNode.Id, len(missingNeedles))
}
if err := c.writeNeedleBlobToTarget(target.location.dataNode.Id, source.info.Id, needleValue, needleBlob); err != nil {
return err
}
}
return nil
}
func (c *commandVolumeCheckDisk) readSourceNeedleBlob(sourceVolumeServer string, volumeId uint32, needleValue needle_map.NeedleValue) (needleBlob []byte, err error) {
err = operation.WithVolumeServerClient(sourceVolumeServer, c.env.option.GrpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
resp, err := client.ReadNeedleBlob(context.Background(), &volume_server_pb.ReadNeedleBlobRequest{
VolumeId: volumeId,
NeedleId: uint64(needleValue.Key),
Offset: needleValue.Offset.ToActualOffset(),
Size: int32(needleValue.Size),
})
if err != nil {
return err
}
needleBlob = resp.NeedleBlob
return nil
})
return
}
func (c *commandVolumeCheckDisk) writeNeedleBlobToTarget(targetVolumeServer string, volumeId uint32, needleValue needle_map.NeedleValue, needleBlob []byte) error {
return operation.WithVolumeServerClient(targetVolumeServer, c.env.option.GrpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
_, err := client.WriteNeedleBlob(context.Background(), &volume_server_pb.WriteNeedleBlobRequest{
VolumeId: volumeId,
NeedleId: uint64(needleValue.Key),
Size: int32(needleValue.Size),
NeedleBlob: needleBlob,
})
return err
})
}
func (c *commandVolumeCheckDisk) readIndexDatabase(db *needle_map.MemDb, collection string, volumeId uint32, volumeServer string, verbose bool, writer io.Writer) error {
var buf bytes.Buffer
if err := c.copyVolumeIndexFile(collection, volumeId, volumeServer, &buf, verbose, writer); err != nil {
return err
}
if verbose {
fmt.Fprintf(writer, "load collection %s volume %d index size %d from %s ...\n", collection, volumeId, buf.Len(), volumeServer)
}
return db.LoadFromReaderAt(bytes.NewReader(buf.Bytes()))
}
func (c *commandVolumeCheckDisk) copyVolumeIndexFile(collection string, volumeId uint32, volumeServer string, buf *bytes.Buffer, verbose bool, writer io.Writer) error {
return operation.WithVolumeServerClient(volumeServer, c.env.option.GrpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
ext := ".idx"
copyFileClient, err := volumeServerClient.CopyFile(context.Background(), &volume_server_pb.CopyFileRequest{
VolumeId: volumeId,
Ext: ".idx",
CompactionRevision: math.MaxUint32,
StopOffset: math.MaxInt64,
Collection: collection,
IsEcVolume: false,
IgnoreSourceFileNotFound: false,
})
if err != nil {
return fmt.Errorf("failed to start copying volume %d%s: %v", volumeId, ext, err)
}
err = writeToBuffer(copyFileClient, buf)
if err != nil {
return fmt.Errorf("failed to copy %d%s from %s: %v", volumeId, ext, volumeServer, err)
}
return nil
})
}
func writeToBuffer(client volume_server_pb.VolumeServer_CopyFileClient, buf *bytes.Buffer) error {
for {
resp, receiveErr := client.Recv()
if receiveErr == io.EOF {
break
}
if receiveErr != nil {
return fmt.Errorf("receiving: %v", receiveErr)
}
buf.Write(resp.FileContent)
}
return nil
}

View file

@ -3,13 +3,12 @@ package needle
import (
"errors"
"fmt"
"io"
"math"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage/backend"
. "github.com/chrislusf/seaweedfs/weed/storage/types"
"github.com/chrislusf/seaweedfs/weed/util"
"io"
"math"
)
const (
@ -156,6 +155,35 @@ func (n *Needle) Append(w backend.BackendStorageFile, version Version) (offset u
return offset, size, actualSize, err
}
func WriteNeedleBlob(w backend.BackendStorageFile, dataSlice []byte, size Size, appendAtNs uint64, version Version) (offset uint64, err error) {
if end, _, e := w.GetStat(); e == nil {
defer func(w backend.BackendStorageFile, off int64) {
if err != nil {
if te := w.Truncate(end); te != nil {
glog.V(0).Infof("Failed to truncate %s back to %d with error: %v", w.Name(), end, te)
}
}
}(w, end)
offset = uint64(end)
} else {
err = fmt.Errorf("Cannot Read Current Volume Position: %v", e)
return
}
if version == Version3 {
tsOffset := NeedleHeaderSize + size + NeedleChecksumSize
util.Uint64toBytes(dataSlice[tsOffset : tsOffset+TimestampSize], appendAtNs)
}
if err == nil {
_, err = w.WriteAt(dataSlice, int64(offset))
}
return
}
func ReadNeedleBlob(r backend.BackendStorageFile, offset int64, size Size, version Version) (dataSlice []byte, err error) {
dataSize := GetActualSize(size, version)

View file

@ -2,6 +2,7 @@ package needle_map
import (
"fmt"
"io"
"os"
"github.com/syndtr/goleveldb/leveldb"
@ -104,7 +105,13 @@ func (cm *MemDb) LoadFromIdx(idxName string) (ret error) {
}
defer idxFile.Close()
return idx.WalkIndexFile(idxFile, func(key NeedleId, offset Offset, size Size) error {
return cm.LoadFromReaderAt(idxFile)
}
func (cm *MemDb) LoadFromReaderAt(readerAt io.ReaderAt) (ret error) {
return idx.WalkIndexFile(readerAt, func(key NeedleId, offset Offset, size Size) error {
if offset.IsZero() || size.IsDeleted() {
return cm.Delete(key)
}

View file

@ -58,6 +58,14 @@ func (v *Volume) readNeedle(n *needle.Needle, readOption *ReadOption) (int, erro
return -1, ErrorNotFound
}
// read fills in Needle content by looking up n.Id from NeedleMapper
func (v *Volume) ReadNeedleBlob(offset int64, size Size) ([]byte, error) {
v.dataFileAccessLock.RLock()
defer v.dataFileAccessLock.RUnlock()
return needle.ReadNeedleBlob(v.DataBackend, offset, size, v.Version())
}
type VolumeFileScanner interface {
VisitSuperBlock(super_block.SuperBlock) error
ReadNeedleBody() bool

View file

@ -299,3 +299,29 @@ func (v *Volume) startWorker() {
}
}()
}
func (v *Volume) WriteNeedleBlob(needleId NeedleId, needleBlob []byte, size Size) error {
v.dataFileAccessLock.Lock()
defer v.dataFileAccessLock.Unlock()
if MaxPossibleVolumeSize < v.nm.ContentSize()+uint64(len(needleBlob)) {
return fmt.Errorf("volume size limit %d exceeded! current size is %d", MaxPossibleVolumeSize, v.nm.ContentSize())
}
appendAtNs := uint64(time.Now().UnixNano())
offset, err := needle.WriteNeedleBlob(v.DataBackend, needleBlob, size, appendAtNs, v.Version())
v.checkReadWriteError(err)
if err != nil {
return err
}
v.lastAppendAtNs = appendAtNs
// add to needle map
if err = v.nm.Put(needleId, ToOffset(int64(offset)), size); err != nil {
glog.V(4).Infof("failed to put in needle map %d: %v", needleId, err)
}
return err
}