avoid reusing context object

fix https://github.com/chrislusf/seaweedfs/issues/1182
This commit is contained in:
Chris Lu 2020-02-25 21:50:12 -08:00
parent bd3254b53f
commit 892e726eb9
86 changed files with 501 additions and 568 deletions

View File

@ -127,7 +127,7 @@ func runBenchmark(cmd *Command, args []string) bool {
defer pprof.StopCPUProfile()
}
b.masterClient = wdclient.NewMasterClient(context.Background(), b.grpcDialOption, "client", strings.Split(*b.masters, ","))
b.masterClient = wdclient.NewMasterClient(b.grpcDialOption, "client", strings.Split(*b.masters, ","))
go b.masterClient.KeepConnectedToMaster()
b.masterClient.WaitUntilConnected()
@ -314,8 +314,8 @@ func readFiles(fileIdLineChan chan string, s *stat) {
}
func grpcFileGet(volumeServer, fid string, grpcDialOption grpc.DialOption) (bytesRead int, err error) {
err = operation.WithVolumeServerClient(volumeServer, grpcDialOption, func(ctx context.Context, client volume_server_pb.VolumeServerClient) error {
fileGetClient, err := client.FileGet(ctx, &volume_server_pb.FileGetRequest{FileId: fid})
err = operation.WithVolumeServerClient(volumeServer, grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
fileGetClient, err := client.FileGet(context.Background(), &volume_server_pb.FileGetRequest{FileId: fid})
if err != nil {
return err
}

View File

@ -161,7 +161,7 @@ func runCopy(cmd *Command, args []string) bool {
}
func readFilerConfiguration(ctx context.Context, grpcDialOption grpc.DialOption, filerGrpcAddress string) (masters []string, collection, replication string, maxMB uint32, err error) {
err = withFilerClient(ctx, filerGrpcAddress, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
err = withFilerClient(filerGrpcAddress, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
resp, err := client.GetFilerConfiguration(ctx, &filer_pb.GetFilerConfigurationRequest{})
if err != nil {
return fmt.Errorf("get filer %s configuration: %v", filerGrpcAddress, err)
@ -257,13 +257,13 @@ func (worker *FileCopyWorker) doEachCopy(ctx context.Context, task FileCopyTask)
}
if chunkCount == 1 {
return worker.uploadFileAsOne(ctx, task, f)
return worker.uploadFileAsOne(task, f)
}
return worker.uploadFileInChunks(ctx, task, f, chunkCount, chunkSize)
return worker.uploadFileInChunks(task, f, chunkCount, chunkSize)
}
func (worker *FileCopyWorker) uploadFileAsOne(ctx context.Context, task FileCopyTask, f *os.File) error {
func (worker *FileCopyWorker) uploadFileAsOne(task FileCopyTask, f *os.File) error {
// upload the file content
fileName := filepath.Base(f.Name())
@ -276,7 +276,7 @@ func (worker *FileCopyWorker) uploadFileAsOne(ctx context.Context, task FileCopy
if task.fileSize > 0 {
// assign a volume
err := withFilerClient(ctx, worker.filerGrpcAddress, worker.options.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
err := withFilerClient(worker.filerGrpcAddress, worker.options.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.AssignVolumeRequest{
Count: 1,
@ -286,7 +286,7 @@ func (worker *FileCopyWorker) uploadFileAsOne(ctx context.Context, task FileCopy
ParentPath: task.destinationUrlPath,
}
assignResult, assignError = client.AssignVolume(ctx, request)
assignResult, assignError = client.AssignVolume(context.Background(), request)
if assignError != nil {
return fmt.Errorf("assign volume failure %v: %v", request, assignError)
}
@ -321,7 +321,7 @@ func (worker *FileCopyWorker) uploadFileAsOne(ctx context.Context, task FileCopy
fmt.Printf("copied %s => http://%s%s%s\n", fileName, worker.filerHost, task.destinationUrlPath, fileName)
}
if err := withFilerClient(ctx, worker.filerGrpcAddress, worker.options.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
if err := withFilerClient(worker.filerGrpcAddress, worker.options.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.CreateEntryRequest{
Directory: task.destinationUrlPath,
Entry: &filer_pb.Entry{
@ -342,7 +342,7 @@ func (worker *FileCopyWorker) uploadFileAsOne(ctx context.Context, task FileCopy
},
}
if err := filer_pb.CreateEntry(ctx, client, request); err != nil {
if err := filer_pb.CreateEntry(client, request); err != nil {
return fmt.Errorf("update fh: %v", err)
}
return nil
@ -353,7 +353,7 @@ func (worker *FileCopyWorker) uploadFileAsOne(ctx context.Context, task FileCopy
return nil
}
func (worker *FileCopyWorker) uploadFileInChunks(ctx context.Context, task FileCopyTask, f *os.File, chunkCount int, chunkSize int64) error {
func (worker *FileCopyWorker) uploadFileInChunks(task FileCopyTask, f *os.File, chunkCount int, chunkSize int64) error {
fileName := filepath.Base(f.Name())
mimeType := detectMimeType(f)
@ -377,7 +377,7 @@ func (worker *FileCopyWorker) uploadFileInChunks(ctx context.Context, task FileC
// assign a volume
var assignResult *filer_pb.AssignVolumeResponse
var assignError error
err := withFilerClient(ctx, worker.filerGrpcAddress, worker.options.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
err := withFilerClient(worker.filerGrpcAddress, worker.options.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.AssignVolumeRequest{
Count: 1,
Replication: *worker.options.replication,
@ -386,7 +386,7 @@ func (worker *FileCopyWorker) uploadFileInChunks(ctx context.Context, task FileC
ParentPath: task.destinationUrlPath,
}
assignResult, assignError = client.AssignVolume(ctx, request)
assignResult, assignError = client.AssignVolume(context.Background(), request)
if assignError != nil {
return fmt.Errorf("assign volume failure %v: %v", request, assignError)
}
@ -449,7 +449,7 @@ func (worker *FileCopyWorker) uploadFileInChunks(ctx context.Context, task FileC
return uploadError
}
if err := withFilerClient(ctx, worker.filerGrpcAddress, worker.options.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
if err := withFilerClient(worker.filerGrpcAddress, worker.options.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.CreateEntryRequest{
Directory: task.destinationUrlPath,
Entry: &filer_pb.Entry{
@ -470,7 +470,7 @@ func (worker *FileCopyWorker) uploadFileInChunks(ctx context.Context, task FileC
},
}
if err := filer_pb.CreateEntry(ctx, client, request); err != nil {
if err := filer_pb.CreateEntry(client, request); err != nil {
return fmt.Errorf("update fh: %v", err)
}
return nil
@ -499,9 +499,9 @@ func detectMimeType(f *os.File) string {
return mimeType
}
func withFilerClient(ctx context.Context, filerAddress string, grpcDialOption grpc.DialOption, fn func(filer_pb.SeaweedFilerClient) error) error {
func withFilerClient(filerAddress string, grpcDialOption grpc.DialOption, fn func(filer_pb.SeaweedFilerClient) error) error {
return util.WithCachedGrpcClient(ctx, func(ctx context.Context, clientConn *grpc.ClientConn) error {
return util.WithCachedGrpcClient(func(clientConn *grpc.ClientConn) error {
client := filer_pb.NewSeaweedFilerClient(clientConn)
return fn(client)
}, filerAddress, grpcDialOption)

View File

@ -128,7 +128,7 @@ func (s3opt *S3Options) startS3Server() bool {
grpcDialOption := security.LoadClientTLS(util.GetViper(), "grpc.client")
ctx := context.Background()
err = withFilerClient(ctx, filerGrpcAddress, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
err = withFilerClient(filerGrpcAddress, grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
resp, err := client.GetFilerConfiguration(ctx, &filer_pb.GetFilerConfigurationRequest{})
if err != nil {
return fmt.Errorf("get filer %s configuration: %v", filerGrpcAddress, err)

View File

@ -37,7 +37,7 @@ type Filer struct {
func NewFiler(masters []string, grpcDialOption grpc.DialOption, bucketFolder string) *Filer {
f := &Filer{
directoryCache: ccache.New(ccache.Configure().MaxSize(1000).ItemsToPrune(100)),
MasterClient: wdclient.NewMasterClient(context.Background(), grpcDialOption, "filer", masters),
MasterClient: wdclient.NewMasterClient(grpcDialOption, "filer", masters),
fileIdDeletionQueue: util.NewUnboundedQueue(),
GrpcDialOption: grpcDialOption,
}

View File

@ -22,10 +22,10 @@ func VolumeId(fileId string) string {
}
type FilerClient interface {
WithFilerClient(ctx context.Context, fn func(context.Context, filer_pb.SeaweedFilerClient) error) error
WithFilerClient(fn func(filer_pb.SeaweedFilerClient) error) error
}
func ReadIntoBuffer(ctx context.Context, filerClient FilerClient, fullFilePath FullPath, buff []byte, chunkViews []*ChunkView, baseOffset int64) (totalRead int64, err error) {
func ReadIntoBuffer(filerClient FilerClient, fullFilePath FullPath, buff []byte, chunkViews []*ChunkView, baseOffset int64) (totalRead int64, err error) {
var vids []string
for _, chunkView := range chunkViews {
vids = append(vids, VolumeId(chunkView.FileId))
@ -33,10 +33,10 @@ func ReadIntoBuffer(ctx context.Context, filerClient FilerClient, fullFilePath F
vid2Locations := make(map[string]*filer_pb.Locations)
err = filerClient.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
err = filerClient.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
glog.V(4).Infof("read fh lookup volume id locations: %v", vids)
resp, err := client.LookupVolume(ctx, &filer_pb.LookupVolumeRequest{
resp, err := client.LookupVolume(context.Background(), &filer_pb.LookupVolumeRequest{
VolumeIds: vids,
})
if err != nil {
@ -93,11 +93,11 @@ func ReadIntoBuffer(ctx context.Context, filerClient FilerClient, fullFilePath F
return
}
func GetEntry(ctx context.Context, filerClient FilerClient, fullFilePath FullPath) (entry *filer_pb.Entry, err error) {
func GetEntry(filerClient FilerClient, fullFilePath FullPath) (entry *filer_pb.Entry, err error) {
dir, name := fullFilePath.DirAndName()
err = filerClient.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
err = filerClient.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.LookupDirectoryEntryRequest{
Directory: dir,
@ -105,7 +105,7 @@ func GetEntry(ctx context.Context, filerClient FilerClient, fullFilePath FullPat
}
// glog.V(3).Infof("read %s request: %v", fullFilePath, request)
resp, err := client.LookupDirectoryEntry(ctx, request)
resp, err := client.LookupDirectoryEntry(context.Background(), request)
if err != nil {
if err == ErrNotFound || strings.Contains(err.Error(), ErrNotFound.Error()) {
return nil
@ -126,9 +126,9 @@ func GetEntry(ctx context.Context, filerClient FilerClient, fullFilePath FullPat
return
}
func ReadDirAllEntries(ctx context.Context, filerClient FilerClient, fullDirPath FullPath, prefix string, fn func(entry *filer_pb.Entry, isLast bool)) (err error) {
func ReadDirAllEntries(filerClient FilerClient, fullDirPath FullPath, prefix string, fn func(entry *filer_pb.Entry, isLast bool)) (err error) {
err = filerClient.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
err = filerClient.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
lastEntryName := ""
@ -140,7 +140,7 @@ func ReadDirAllEntries(ctx context.Context, filerClient FilerClient, fullDirPath
}
glog.V(3).Infof("read directory: %v", request)
stream, err := client.ListEntries(ctx, request)
stream, err := client.ListEntries(context.Background(), request)
if err != nil {
return fmt.Errorf("list %s: %v", fullDirPath, err)
}

View File

@ -45,7 +45,7 @@ func (f *Filer) DeleteEntryMetaAndData(ctx context.Context, p FullPath, isRecurs
}
if isCollection {
collectionName := entry.Name()
f.doDeleteCollection(ctx, collectionName)
f.doDeleteCollection(collectionName)
f.deleteBucket(collectionName)
}
@ -110,10 +110,10 @@ func (f *Filer) doDeleteEntryMetaAndData(ctx context.Context, entry *Entry, shou
return nil
}
func (f *Filer) doDeleteCollection(ctx context.Context, collectionName string) (err error) {
func (f *Filer) doDeleteCollection(collectionName string) (err error) {
return f.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
_, err := client.CollectionDelete(ctx, &master_pb.CollectionDeleteRequest{
return f.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
_, err := client.CollectionDelete(context.Background(), &master_pb.CollectionDeleteRequest{
Name: collectionName,
})
if err != nil {

View File

@ -126,10 +126,10 @@ func (dir *Dir) Create(ctx context.Context, req *fuse.CreateRequest,
},
OExcl: req.Flags&fuse.OpenExclusive != 0,
}
glog.V(1).Infof("create: %v", req.String())
glog.V(1).Infof("create %s/%s: %v", dir.Path, req.Name, req.Flags)
if err := dir.wfs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
if err := filer_pb.CreateEntry(ctx, client, request); err != nil {
if err := dir.wfs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
if err := filer_pb.CreateEntry(client, request); err != nil {
if strings.Contains(err.Error(), "EEXIST") {
return fuse.EEXIST
}
@ -167,7 +167,7 @@ func (dir *Dir) Mkdir(ctx context.Context, req *fuse.MkdirRequest) (fs.Node, err
},
}
err := dir.wfs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
err := dir.wfs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.CreateEntryRequest{
Directory: dir.Path,
@ -175,7 +175,7 @@ func (dir *Dir) Mkdir(ctx context.Context, req *fuse.MkdirRequest) (fs.Node, err
}
glog.V(1).Infof("mkdir: %v", request)
if err := filer_pb.CreateEntry(ctx, client, request); err != nil {
if err := filer_pb.CreateEntry(client, request); err != nil {
glog.V(0).Infof("mkdir %s/%s: %v", dir.Path, req.Name, err)
return err
}
@ -200,7 +200,7 @@ func (dir *Dir) Lookup(ctx context.Context, req *fuse.LookupRequest, resp *fuse.
if entry == nil {
// glog.V(3).Infof("dir Lookup cache miss %s", fullFilePath)
entry, err = filer2.GetEntry(ctx, dir.wfs, fullFilePath)
entry, err = filer2.GetEntry(dir.wfs, fullFilePath)
if err != nil {
glog.V(1).Infof("dir GetEntry %s: %v", fullFilePath, err)
return nil, fuse.ENOENT
@ -239,7 +239,7 @@ func (dir *Dir) ReadDirAll(ctx context.Context) (ret []fuse.Dirent, err error) {
cacheTtl := 5 * time.Minute
readErr := filer2.ReadDirAllEntries(ctx, dir.wfs, filer2.FullPath(dir.Path), "", func(entry *filer_pb.Entry, isLast bool) {
readErr := filer2.ReadDirAllEntries(dir.wfs, filer2.FullPath(dir.Path), "", func(entry *filer_pb.Entry, isLast bool) {
fullpath := filer2.NewFullPath(dir.Path, entry.Name)
inode := fullpath.AsInode()
if entry.IsDirectory {
@ -262,17 +262,17 @@ func (dir *Dir) ReadDirAll(ctx context.Context) (ret []fuse.Dirent, err error) {
func (dir *Dir) Remove(ctx context.Context, req *fuse.RemoveRequest) error {
if !req.Dir {
return dir.removeOneFile(ctx, req)
return dir.removeOneFile(req)
}
return dir.removeFolder(ctx, req)
return dir.removeFolder(req)
}
func (dir *Dir) removeOneFile(ctx context.Context, req *fuse.RemoveRequest) error {
func (dir *Dir) removeOneFile(req *fuse.RemoveRequest) error {
filePath := filer2.NewFullPath(dir.Path, req.Name)
entry, err := filer2.GetEntry(ctx, dir.wfs, filePath)
entry, err := filer2.GetEntry(dir.wfs, filePath)
if err != nil {
return err
}
@ -280,11 +280,11 @@ func (dir *Dir) removeOneFile(ctx context.Context, req *fuse.RemoveRequest) erro
return nil
}
dir.wfs.deleteFileChunks(ctx, entry.Chunks)
dir.wfs.deleteFileChunks(entry.Chunks)
dir.wfs.cacheDelete(filePath)
return dir.wfs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
return dir.wfs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.DeleteEntryRequest{
Directory: dir.Path,
@ -293,7 +293,7 @@ func (dir *Dir) removeOneFile(ctx context.Context, req *fuse.RemoveRequest) erro
}
glog.V(3).Infof("remove file: %v", request)
_, err := client.DeleteEntry(ctx, request)
_, err := client.DeleteEntry(context.Background(), request)
if err != nil {
glog.V(3).Infof("not found remove file %s/%s: %v", dir.Path, req.Name, err)
return fuse.ENOENT
@ -304,11 +304,11 @@ func (dir *Dir) removeOneFile(ctx context.Context, req *fuse.RemoveRequest) erro
}
func (dir *Dir) removeFolder(ctx context.Context, req *fuse.RemoveRequest) error {
func (dir *Dir) removeFolder(req *fuse.RemoveRequest) error {
dir.wfs.cacheDelete(filer2.NewFullPath(dir.Path, req.Name))
return dir.wfs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
return dir.wfs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.DeleteEntryRequest{
Directory: dir.Path,
@ -317,7 +317,7 @@ func (dir *Dir) removeFolder(ctx context.Context, req *fuse.RemoveRequest) error
}
glog.V(3).Infof("remove directory entry: %v", request)
_, err := client.DeleteEntry(ctx, request)
_, err := client.DeleteEntry(context.Background(), request)
if err != nil {
glog.V(3).Infof("not found remove %s/%s: %v", dir.Path, req.Name, err)
return fuse.ENOENT
@ -419,7 +419,7 @@ func (dir *Dir) Forget() {
func (dir *Dir) maybeLoadEntry(ctx context.Context) error {
if dir.entry == nil {
parentDirPath, name := filer2.FullPath(dir.Path).DirAndName()
entry, err := dir.wfs.maybeLoadEntry(ctx, parentDirPath, name)
entry, err := dir.wfs.maybeLoadEntry(parentDirPath, name)
if err != nil {
return err
}
@ -432,7 +432,7 @@ func (dir *Dir) saveEntry(ctx context.Context) error {
parentDir, name := filer2.FullPath(dir.Path).DirAndName()
return dir.wfs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
return dir.wfs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.UpdateEntryRequest{
Directory: parentDir,
@ -440,7 +440,7 @@ func (dir *Dir) saveEntry(ctx context.Context) error {
}
glog.V(1).Infof("save dir entry: %v", request)
_, err := client.UpdateEntry(ctx, request)
_, err := client.UpdateEntry(context.Background(), request)
if err != nil {
glog.V(0).Infof("UpdateEntry dir %s/%s: %v", parentDir, name, err)
return fuse.EIO

View File

@ -35,8 +35,8 @@ func (dir *Dir) Symlink(ctx context.Context, req *fuse.SymlinkRequest) (fs.Node,
},
}
err := dir.wfs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
if err := filer_pb.CreateEntry(ctx, client, request); err != nil {
err := dir.wfs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
if err := filer_pb.CreateEntry(client, request); err != nil {
glog.V(0).Infof("symlink %s/%s: %v", dir.Path, req.NewName, err)
return fuse.EIO
}

View File

@ -15,7 +15,7 @@ func (dir *Dir) Rename(ctx context.Context, req *fuse.RenameRequest, newDirector
newDir := newDirectory.(*Dir)
glog.V(4).Infof("dir Rename %s/%s => %s/%s", dir.Path, req.OldName, newDir.Path, req.NewName)
err := dir.wfs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
err := dir.wfs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.AtomicRenameEntryRequest{
OldDirectory: dir.Path,
@ -24,7 +24,7 @@ func (dir *Dir) Rename(ctx context.Context, req *fuse.RenameRequest, newDirector
NewName: req.NewName,
}
_, err := client.AtomicRenameEntry(ctx, request)
_, err := client.AtomicRenameEntry(context.Background(), request)
if err != nil {
glog.V(0).Infof("dir Rename %s/%s => %s/%s : %v", dir.Path, req.OldName, newDir.Path, req.NewName, err)
return fuse.EIO

View File

@ -52,7 +52,7 @@ func (pages *ContinuousDirtyPages) AddPage(ctx context.Context, offset int64, da
var hasSavedData bool
if pages.intervals.TotalSize() > pages.f.wfs.option.ChunkSizeLimit {
chunk, hasSavedData, err = pages.saveExistingLargestPageToStorage(ctx)
chunk, hasSavedData, err = pages.saveExistingLargestPageToStorage()
if hasSavedData {
chunks = append(chunks, chunk)
}
@ -67,7 +67,7 @@ func (pages *ContinuousDirtyPages) flushAndSave(ctx context.Context, offset int6
var newChunks []*filer_pb.FileChunk
// flush existing
if newChunks, err = pages.saveExistingPagesToStorage(ctx); err == nil {
if newChunks, err = pages.saveExistingPagesToStorage(); err == nil {
if newChunks != nil {
chunks = append(chunks, newChunks...)
}
@ -76,7 +76,7 @@ func (pages *ContinuousDirtyPages) flushAndSave(ctx context.Context, offset int6
}
// flush the new page
if chunk, err = pages.saveToStorage(ctx, bytes.NewReader(data), offset, int64(len(data))); err == nil {
if chunk, err = pages.saveToStorage(bytes.NewReader(data), offset, int64(len(data))); err == nil {
if chunk != nil {
glog.V(4).Infof("%s/%s flush big request [%d,%d) to %s", pages.f.dir.Path, pages.f.Name, chunk.Offset, chunk.Offset+int64(chunk.Size), chunk.FileId)
chunks = append(chunks, chunk)
@ -89,22 +89,22 @@ func (pages *ContinuousDirtyPages) flushAndSave(ctx context.Context, offset int6
return
}
func (pages *ContinuousDirtyPages) FlushToStorage(ctx context.Context) (chunks []*filer_pb.FileChunk, err error) {
func (pages *ContinuousDirtyPages) FlushToStorage() (chunks []*filer_pb.FileChunk, err error) {
pages.lock.Lock()
defer pages.lock.Unlock()
return pages.saveExistingPagesToStorage(ctx)
return pages.saveExistingPagesToStorage()
}
func (pages *ContinuousDirtyPages) saveExistingPagesToStorage(ctx context.Context) (chunks []*filer_pb.FileChunk, err error) {
func (pages *ContinuousDirtyPages) saveExistingPagesToStorage() (chunks []*filer_pb.FileChunk, err error) {
var hasSavedData bool
var chunk *filer_pb.FileChunk
for {
chunk, hasSavedData, err = pages.saveExistingLargestPageToStorage(ctx)
chunk, hasSavedData, err = pages.saveExistingLargestPageToStorage()
if !hasSavedData {
return chunks, err
}
@ -118,14 +118,14 @@ func (pages *ContinuousDirtyPages) saveExistingPagesToStorage(ctx context.Contex
}
func (pages *ContinuousDirtyPages) saveExistingLargestPageToStorage(ctx context.Context) (chunk *filer_pb.FileChunk, hasSavedData bool, err error) {
func (pages *ContinuousDirtyPages) saveExistingLargestPageToStorage() (chunk *filer_pb.FileChunk, hasSavedData bool, err error) {
maxList := pages.intervals.RemoveLargestIntervalLinkedList()
if maxList == nil {
return nil, false, nil
}
chunk, err = pages.saveToStorage(ctx, maxList.ToReader(), maxList.Offset(), maxList.Size())
chunk, err = pages.saveToStorage(maxList.ToReader(), maxList.Offset(), maxList.Size())
if err == nil {
hasSavedData = true
glog.V(3).Infof("%s saveToStorage [%d,%d) %s", pages.f.fullpath(), maxList.Offset(), maxList.Offset()+maxList.Size(), chunk.FileId)
@ -137,14 +137,14 @@ func (pages *ContinuousDirtyPages) saveExistingLargestPageToStorage(ctx context.
return
}
func (pages *ContinuousDirtyPages) saveToStorage(ctx context.Context, reader io.Reader, offset int64, size int64) (*filer_pb.FileChunk, error) {
func (pages *ContinuousDirtyPages) saveToStorage(reader io.Reader, offset int64, size int64) (*filer_pb.FileChunk, error) {
var fileId, host string
var auth security.EncodedJwt
dir, _ := pages.f.fullpath().DirAndName()
if err := pages.f.wfs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
if err := pages.f.wfs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.AssignVolumeRequest{
Count: 1,
@ -155,7 +155,7 @@ func (pages *ContinuousDirtyPages) saveToStorage(ctx context.Context, reader io.
ParentPath: dir,
}
resp, err := client.AssignVolume(ctx, request)
resp, err := client.AssignVolume(context.Background(), request)
if err != nil {
glog.V(0).Infof("assign volume failure %v: %v", request, err)
return err

View File

@ -148,7 +148,7 @@ func (file *File) Setattr(ctx context.Context, req *fuse.SetattrRequest, resp *f
file.wfs.cacheDelete(file.fullpath())
return file.saveEntry(ctx)
return file.saveEntry()
}
@ -166,7 +166,7 @@ func (file *File) Setxattr(ctx context.Context, req *fuse.SetxattrRequest) error
file.wfs.cacheDelete(file.fullpath())
return file.saveEntry(ctx)
return file.saveEntry()
}
@ -184,7 +184,7 @@ func (file *File) Removexattr(ctx context.Context, req *fuse.RemovexattrRequest)
file.wfs.cacheDelete(file.fullpath())
return file.saveEntry(ctx)
return file.saveEntry()
}
@ -221,7 +221,7 @@ func (file *File) Forget() {
func (file *File) maybeLoadEntry(ctx context.Context) error {
if file.entry == nil || file.isOpen <= 0 {
entry, err := file.wfs.maybeLoadEntry(ctx, file.dir.Path, file.Name)
entry, err := file.wfs.maybeLoadEntry(file.dir.Path, file.Name)
if err != nil {
return err
}
@ -256,8 +256,8 @@ func (file *File) setEntry(entry *filer_pb.Entry) {
file.entryViewCache = filer2.NonOverlappingVisibleIntervals(file.entry.Chunks)
}
func (file *File) saveEntry(ctx context.Context) error {
return file.wfs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
func (file *File) saveEntry() error {
return file.wfs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.UpdateEntryRequest{
Directory: file.dir.Path,
@ -265,7 +265,7 @@ func (file *File) saveEntry(ctx context.Context) error {
}
glog.V(1).Infof("save file entry: %v", request)
_, err := client.UpdateEntry(ctx, request)
_, err := client.UpdateEntry(context.Background(), request)
if err != nil {
glog.V(0).Infof("UpdateEntry file %s/%s: %v", file.dir.Path, file.Name, err)
return fuse.EIO

View File

@ -89,7 +89,7 @@ func (fh *FileHandle) readFromChunks(ctx context.Context, buff []byte, offset in
chunkViews := filer2.ViewFromVisibleIntervals(fh.f.entryViewCache, offset, len(buff))
totalRead, err := filer2.ReadIntoBuffer(ctx, fh.f.wfs, fh.f.fullpath(), buff, chunkViews, offset)
totalRead, err := filer2.ReadIntoBuffer(fh.f.wfs, fh.f.fullpath(), buff, chunkViews, offset)
if err != nil {
glog.Errorf("file handle read %s: %v", fh.f.fullpath(), err)
@ -154,7 +154,7 @@ func (fh *FileHandle) Flush(ctx context.Context, req *fuse.FlushRequest) error {
// send the data to the OS
glog.V(4).Infof("%s fh %d flush %v", fh.f.fullpath(), fh.handle, req)
chunks, err := fh.dirtyPages.FlushToStorage(ctx)
chunks, err := fh.dirtyPages.FlushToStorage()
if err != nil {
glog.Errorf("flush %s: %v", fh.f.fullpath(), err)
return fuse.EIO
@ -169,7 +169,7 @@ func (fh *FileHandle) Flush(ctx context.Context, req *fuse.FlushRequest) error {
return nil
}
err = fh.f.wfs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
err = fh.f.wfs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
if fh.f.entry.Attributes != nil {
fh.f.entry.Attributes.Mime = fh.contentType
@ -196,12 +196,12 @@ func (fh *FileHandle) Flush(ctx context.Context, req *fuse.FlushRequest) error {
fh.f.entry.Chunks = chunks
// fh.f.entryViewCache = nil
if err := filer_pb.CreateEntry(ctx, client, request); err != nil {
glog.Errorf("update fh: %v", err)
return fmt.Errorf("update fh: %v", err)
if err := filer_pb.CreateEntry(client, request); err != nil {
glog.Errorf("fh flush create %s: %v", fh.f.fullpath(), err)
return fmt.Errorf("fh flush create %s: %v", fh.f.fullpath(), err)
}
fh.f.wfs.deleteFileChunks(ctx, garbages)
fh.f.wfs.deleteFileChunks(garbages)
for i, chunk := range garbages {
glog.V(3).Infof("garbage %s chunks %d: %v [%d,%d)", fh.f.fullpath(), i, chunk.FileId, chunk.Offset, chunk.Offset+int64(chunk.Size))
}

View File

@ -5,7 +5,6 @@ import (
"fmt"
"math"
"os"
"strings"
"sync"
"time"
@ -88,23 +87,16 @@ func (wfs *WFS) Root() (fs.Node, error) {
return wfs.root, nil
}
func (wfs *WFS) WithFilerClient(ctx context.Context, fn func(context.Context, filer_pb.SeaweedFilerClient) error) error {
func (wfs *WFS) WithFilerClient(fn func(filer_pb.SeaweedFilerClient) error) error {
err := util.WithCachedGrpcClient(ctx, func(ctx2 context.Context, grpcConnection *grpc.ClientConn) error {
err := util.WithCachedGrpcClient(func(grpcConnection *grpc.ClientConn) error {
client := filer_pb.NewSeaweedFilerClient(grpcConnection)
return fn(ctx2, client)
return fn(client)
}, wfs.option.FilerGrpcAddress, wfs.option.GrpcDialOption)
if err == nil {
return nil
}
if strings.Contains(err.Error(), "context canceled") {
glog.V(0).Infoln("retry context canceled request...")
return util.WithCachedGrpcClient(context.Background(), func(ctx2 context.Context, grpcConnection *grpc.ClientConn) error {
client := filer_pb.NewSeaweedFilerClient(grpcConnection)
return fn(ctx2, client)
}, wfs.option.FilerGrpcAddress, wfs.option.GrpcDialOption)
}
return err
}
@ -162,7 +154,7 @@ func (wfs *WFS) Statfs(ctx context.Context, req *fuse.StatfsRequest, resp *fuse.
if wfs.stats.lastChecked < time.Now().Unix()-20 {
err := wfs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
err := wfs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.StatisticsRequest{
Collection: wfs.option.Collection,
@ -171,7 +163,7 @@ func (wfs *WFS) Statfs(ctx context.Context, req *fuse.StatfsRequest, resp *fuse.
}
glog.V(4).Infof("reading filer stats: %+v", request)
resp, err := client.Statistics(ctx, request)
resp, err := client.Statistics(context.Background(), request)
if err != nil {
glog.V(0).Infof("reading filer stats %v: %v", request, err)
return err

View File

@ -10,7 +10,7 @@ import (
"google.golang.org/grpc"
)
func (wfs *WFS) deleteFileChunks(ctx context.Context, chunks []*filer_pb.FileChunk) {
func (wfs *WFS) deleteFileChunks(chunks []*filer_pb.FileChunk) {
if len(chunks) == 0 {
return
}
@ -20,13 +20,13 @@ func (wfs *WFS) deleteFileChunks(ctx context.Context, chunks []*filer_pb.FileChu
fileIds = append(fileIds, chunk.GetFileIdString())
}
wfs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
deleteFileIds(ctx, wfs.option.GrpcDialOption, client, fileIds)
wfs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
deleteFileIds(wfs.option.GrpcDialOption, client, fileIds)
return nil
})
}
func deleteFileIds(ctx context.Context, grpcDialOption grpc.DialOption, client filer_pb.SeaweedFilerClient, fileIds []string) error {
func deleteFileIds(grpcDialOption grpc.DialOption, client filer_pb.SeaweedFilerClient, fileIds []string) error {
var vids []string
for _, fileId := range fileIds {
@ -38,7 +38,7 @@ func deleteFileIds(ctx context.Context, grpcDialOption grpc.DialOption, client f
m := make(map[string]operation.LookupResult)
glog.V(4).Infof("remove file lookup volume id locations: %v", vids)
resp, err := client.LookupVolume(ctx, &filer_pb.LookupVolumeRequest{
resp, err := client.LookupVolume(context.Background(), &filer_pb.LookupVolumeRequest{
VolumeIds: vids,
})
if err != nil {

View File

@ -108,7 +108,7 @@ func listxattr(entry *filer_pb.Entry, req *fuse.ListxattrRequest, resp *fuse.Lis
}
func (wfs *WFS) maybeLoadEntry(ctx context.Context, dir, name string) (entry *filer_pb.Entry, err error) {
func (wfs *WFS) maybeLoadEntry(dir, name string) (entry *filer_pb.Entry, err error) {
fullpath := filer2.NewFullPath(dir, name)
entry = wfs.cacheGet(fullpath)
@ -117,14 +117,14 @@ func (wfs *WFS) maybeLoadEntry(ctx context.Context, dir, name string) (entry *fi
}
// glog.V(3).Infof("read entry cache miss %s", fullpath)
err = wfs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
err = wfs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.LookupDirectoryEntryRequest{
Name: name,
Directory: dir,
}
resp, err := client.LookupDirectoryEntry(ctx, request)
resp, err := client.LookupDirectoryEntry(context.Background(), request)
if err != nil || resp == nil || resp.Entry == nil {
if err == filer2.ErrNotFound || strings.Contains(err.Error(), filer2.ErrNotFound.Error()) {
glog.V(3).Infof("file attr read not found file %v: %v", request, err)

View File

@ -46,7 +46,7 @@ func Assign(server string, grpcDialOption grpc.DialOption, primaryRequest *Volum
continue
}
lastError = WithMasterServerClient(server, grpcDialOption, func(ctx context.Context, masterClient master_pb.SeaweedClient) error {
lastError = WithMasterServerClient(server, grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
req := &master_pb.AssignRequest{
Count: primaryRequest.Count,

View File

@ -10,7 +10,6 @@ import (
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
)
@ -109,15 +108,13 @@ func DeleteFilesWithLookupVolumeId(grpcDialOption grpc.DialOption, fileIds []str
ret = append(ret, result...)
}
glog.V(1).Infof("deleted %d items", len(ret))
return ret, err
}
// DeleteFilesAtOneVolumeServer deletes a list of files that is on one volume server via gRpc
func DeleteFilesAtOneVolumeServer(volumeServer string, grpcDialOption grpc.DialOption, fileIds []string) (ret []*volume_server_pb.DeleteResult, err error) {
err = WithVolumeServerClient(volumeServer, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
err = WithVolumeServerClient(volumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
req := &volume_server_pb.BatchDeleteRequest{
FileIds: fileIds,

View File

@ -1,29 +1,28 @@
package operation
import (
"context"
"fmt"
"strconv"
"strings"
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/util"
"google.golang.org/grpc"
"strconv"
"strings"
)
func WithVolumeServerClient(volumeServer string, grpcDialOption grpc.DialOption, fn func(context.Context, volume_server_pb.VolumeServerClient) error) error {
ctx := context.Background()
func WithVolumeServerClient(volumeServer string, grpcDialOption grpc.DialOption, fn func(volume_server_pb.VolumeServerClient) error) error {
grpcAddress, err := toVolumeServerGrpcAddress(volumeServer)
if err != nil {
return err
}
return util.WithCachedGrpcClient(ctx, func(ctx2 context.Context, grpcConnection *grpc.ClientConn) error {
return util.WithCachedGrpcClient(func(grpcConnection *grpc.ClientConn) error {
client := volume_server_pb.NewVolumeServerClient(grpcConnection)
return fn(ctx2, client)
return fn(client)
}, grpcAddress, grpcDialOption)
}
@ -38,18 +37,16 @@ func toVolumeServerGrpcAddress(volumeServer string) (grpcAddress string, err err
return fmt.Sprintf("%s:%d", volumeServer[0:sepIndex], port+10000), nil
}
func WithMasterServerClient(masterServer string, grpcDialOption grpc.DialOption, fn func(ctx2 context.Context, masterClient master_pb.SeaweedClient) error) error {
ctx := context.Background()
func WithMasterServerClient(masterServer string, grpcDialOption grpc.DialOption, fn func(masterClient master_pb.SeaweedClient) error) error {
masterGrpcAddress, parseErr := util.ParseServerToGrpcAddress(masterServer)
if parseErr != nil {
return fmt.Errorf("failed to parse master grpc %v: %v", masterServer, parseErr)
}
return util.WithCachedGrpcClient(ctx, func(ctx2 context.Context, grpcConnection *grpc.ClientConn) error {
return util.WithCachedGrpcClient(func(grpcConnection *grpc.ClientConn) error {
client := master_pb.NewSeaweedClient(grpcConnection)
return fn(ctx2, client)
return fn(client)
}, masterGrpcAddress, grpcDialOption)
}

View File

@ -99,12 +99,12 @@ func LookupVolumeIds(server string, grpcDialOption grpc.DialOption, vids []strin
//only query unknown_vids
err := WithMasterServerClient(server, grpcDialOption, func(ctx context.Context, masterClient master_pb.SeaweedClient) error {
err := WithMasterServerClient(server, grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
req := &master_pb.LookupVolumeRequest{
VolumeIds: unknown_vids,
}
resp, grpcErr := masterClient.LookupVolume(ctx, req)
resp, grpcErr := masterClient.LookupVolume(context.Background(), req)
if grpcErr != nil {
return grpcErr
}

View File

@ -9,9 +9,9 @@ import (
func Statistics(server string, grpcDialOption grpc.DialOption, req *master_pb.StatisticsRequest) (resp *master_pb.StatisticsResponse, err error) {
err = WithMasterServerClient(server, grpcDialOption, func(ctx context.Context, masterClient master_pb.SeaweedClient) error {
err = WithMasterServerClient(server, grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
grpcResponse, grpcErr := masterClient.Statistics(ctx, req)
grpcResponse, grpcErr := masterClient.Statistics(context.Background(), req)
if grpcErr != nil {
return grpcErr
}

View File

@ -8,9 +8,9 @@ import (
func GetVolumeSyncStatus(server string, grpcDialOption grpc.DialOption, vid uint32) (resp *volume_server_pb.VolumeSyncStatusResponse, err error) {
WithVolumeServerClient(server, grpcDialOption, func(ctx context.Context, client volume_server_pb.VolumeServerClient) error {
WithVolumeServerClient(server, grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
resp, err = client.VolumeSyncStatus(ctx, &volume_server_pb.VolumeSyncStatusRequest{
resp, err = client.VolumeSyncStatus(context.Background(), &volume_server_pb.VolumeSyncStatusRequest{
VolumeId: vid,
})
return nil

View File

@ -5,9 +5,10 @@ import (
"fmt"
"io"
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
"google.golang.org/grpc"
)
func TailVolume(master string, grpcDialOption grpc.DialOption, vid needle.VolumeId, sinceNs uint64, timeoutSeconds int, fn func(n *needle.Needle) error) error {
@ -26,9 +27,9 @@ func TailVolume(master string, grpcDialOption grpc.DialOption, vid needle.Volume
}
func TailVolumeFromSource(volumeServer string, grpcDialOption grpc.DialOption, vid needle.VolumeId, sinceNs uint64, idleTimeoutSeconds int, fn func(n *needle.Needle) error) error {
return WithVolumeServerClient(volumeServer, grpcDialOption, func(ctx context.Context, client volume_server_pb.VolumeServerClient) error {
return WithVolumeServerClient(volumeServer, grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
stream, err := client.VolumeTailSender(ctx, &volume_server_pb.VolumeTailSenderRequest{
stream, err := client.VolumeTailSender(context.Background(), &volume_server_pb.VolumeTailSenderRequest{
VolumeId: uint32(vid),
SinceNs: sinceNs,
IdleTimeoutSeconds: uint32(idleTimeoutSeconds),

View File

@ -4,6 +4,7 @@ import (
"context"
"fmt"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
)
@ -71,13 +72,15 @@ func AfterEntryDeserialization(chunks []*FileChunk) {
}
}
func CreateEntry(ctx context.Context, client SeaweedFilerClient, request *CreateEntryRequest) error {
resp, err := client.CreateEntry(ctx, request)
func CreateEntry(client SeaweedFilerClient, request *CreateEntryRequest) error {
resp, err := client.CreateEntry(context.Background(), request)
if err != nil {
glog.V(1).Infof("create entry %s/%s %v: %v", request.Directory, request.Entry.Name, request.OExcl, err)
return fmt.Errorf("CreateEntry: %v", err)
}
if resp.Error != "" {
glog.V(1).Infof("create entry %s/%s %v: %v", request.Directory, request.Entry.Name, request.OExcl, err)
return fmt.Errorf("CreateEntry : %v", resp.Error)
}
return err
return nil
}

View File

@ -41,28 +41,28 @@ func (r *Replicator) Replicate(ctx context.Context, key string, message *filer_p
key = newKey
if message.OldEntry != nil && message.NewEntry == nil {
glog.V(4).Infof("deleting %v", key)
return r.sink.DeleteEntry(ctx, key, message.OldEntry.IsDirectory, message.DeleteChunks)
return r.sink.DeleteEntry(key, message.OldEntry.IsDirectory, message.DeleteChunks)
}
if message.OldEntry == nil && message.NewEntry != nil {
glog.V(4).Infof("creating %v", key)
return r.sink.CreateEntry(ctx, key, message.NewEntry)
return r.sink.CreateEntry(key, message.NewEntry)
}
if message.OldEntry == nil && message.NewEntry == nil {
glog.V(0).Infof("weird message %+v", message)
return nil
}
foundExisting, err := r.sink.UpdateEntry(ctx, key, message.OldEntry, message.NewParentPath, message.NewEntry, message.DeleteChunks)
foundExisting, err := r.sink.UpdateEntry(key, message.OldEntry, message.NewParentPath, message.NewEntry, message.DeleteChunks)
if foundExisting {
glog.V(4).Infof("updated %v", key)
return err
}
err = r.sink.DeleteEntry(ctx, key, message.OldEntry.IsDirectory, false)
err = r.sink.DeleteEntry(key, message.OldEntry.IsDirectory, false)
if err != nil {
return fmt.Errorf("delete old entry %v: %v", key, err)
}
glog.V(4).Infof("creating missing %v", key)
return r.sink.CreateEntry(ctx, key, message.NewEntry)
return r.sink.CreateEntry(key, message.NewEntry)
}

View File

@ -70,7 +70,7 @@ func (g *AzureSink) initialize(accountName, accountKey, container, dir string) e
return nil
}
func (g *AzureSink) DeleteEntry(ctx context.Context, key string, isDirectory, deleteIncludeChunks bool) error {
func (g *AzureSink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool) error {
key = cleanKey(key)
@ -78,7 +78,7 @@ func (g *AzureSink) DeleteEntry(ctx context.Context, key string, isDirectory, de
key = key + "/"
}
if _, err := g.containerURL.NewBlobURL(key).Delete(ctx,
if _, err := g.containerURL.NewBlobURL(key).Delete(context.Background(),
azblob.DeleteSnapshotsOptionInclude, azblob.BlobAccessConditions{}); err != nil {
return fmt.Errorf("azure delete %s/%s: %v", g.container, key, err)
}
@ -87,7 +87,7 @@ func (g *AzureSink) DeleteEntry(ctx context.Context, key string, isDirectory, de
}
func (g *AzureSink) CreateEntry(ctx context.Context, key string, entry *filer_pb.Entry) error {
func (g *AzureSink) CreateEntry(key string, entry *filer_pb.Entry) error {
key = cleanKey(key)
@ -102,21 +102,21 @@ func (g *AzureSink) CreateEntry(ctx context.Context, key string, entry *filer_pb
// Azure Storage account's container.
appendBlobURL := g.containerURL.NewAppendBlobURL(key)
_, err := appendBlobURL.Create(ctx, azblob.BlobHTTPHeaders{}, azblob.Metadata{}, azblob.BlobAccessConditions{})
_, err := appendBlobURL.Create(context.Background(), azblob.BlobHTTPHeaders{}, azblob.Metadata{}, azblob.BlobAccessConditions{})
if err != nil {
return err
}
for _, chunk := range chunkViews {
fileUrl, err := g.filerSource.LookupFileId(ctx, chunk.FileId)
fileUrl, err := g.filerSource.LookupFileId(chunk.FileId)
if err != nil {
return err
}
var writeErr error
_, readErr := util.ReadUrlAsStream(fileUrl, chunk.Offset, int(chunk.Size), func(data []byte) {
_, writeErr = appendBlobURL.AppendBlock(ctx, bytes.NewReader(data), azblob.AppendBlobAccessConditions{}, nil)
_, writeErr = appendBlobURL.AppendBlock(context.Background(), bytes.NewReader(data), azblob.AppendBlobAccessConditions{}, nil)
})
if readErr != nil {
@ -132,7 +132,7 @@ func (g *AzureSink) CreateEntry(ctx context.Context, key string, entry *filer_pb
}
func (g *AzureSink) UpdateEntry(ctx context.Context, key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
func (g *AzureSink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
key = cleanKey(key)
// TODO improve efficiency
return false, nil

View File

@ -58,7 +58,7 @@ func (g *B2Sink) initialize(accountId, accountKey, bucket, dir string) error {
return nil
}
func (g *B2Sink) DeleteEntry(ctx context.Context, key string, isDirectory, deleteIncludeChunks bool) error {
func (g *B2Sink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool) error {
key = cleanKey(key)
@ -66,18 +66,18 @@ func (g *B2Sink) DeleteEntry(ctx context.Context, key string, isDirectory, delet
key = key + "/"
}
bucket, err := g.client.Bucket(ctx, g.bucket)
bucket, err := g.client.Bucket(context.Background(), g.bucket)
if err != nil {
return err
}
targetObject := bucket.Object(key)
return targetObject.Delete(ctx)
return targetObject.Delete(context.Background())
}
func (g *B2Sink) CreateEntry(ctx context.Context, key string, entry *filer_pb.Entry) error {
func (g *B2Sink) CreateEntry(key string, entry *filer_pb.Entry) error {
key = cleanKey(key)
@ -88,17 +88,17 @@ func (g *B2Sink) CreateEntry(ctx context.Context, key string, entry *filer_pb.En
totalSize := filer2.TotalSize(entry.Chunks)
chunkViews := filer2.ViewFromChunks(entry.Chunks, 0, int(totalSize))
bucket, err := g.client.Bucket(ctx, g.bucket)
bucket, err := g.client.Bucket(context.Background(), g.bucket)
if err != nil {
return err
}
targetObject := bucket.Object(key)
writer := targetObject.NewWriter(ctx)
writer := targetObject.NewWriter(context.Background())
for _, chunk := range chunkViews {
fileUrl, err := g.filerSource.LookupFileId(ctx, chunk.FileId)
fileUrl, err := g.filerSource.LookupFileId(chunk.FileId)
if err != nil {
return err
}
@ -124,7 +124,7 @@ func (g *B2Sink) CreateEntry(ctx context.Context, key string, entry *filer_pb.En
}
func (g *B2Sink) UpdateEntry(ctx context.Context, key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
func (g *B2Sink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
key = cleanKey(key)

View File

@ -15,7 +15,7 @@ import (
"github.com/chrislusf/seaweedfs/weed/util"
)
func (fs *FilerSink) replicateChunks(ctx context.Context, sourceChunks []*filer_pb.FileChunk, dir string) (replicatedChunks []*filer_pb.FileChunk, err error) {
func (fs *FilerSink) replicateChunks(sourceChunks []*filer_pb.FileChunk, dir string) (replicatedChunks []*filer_pb.FileChunk, err error) {
if len(sourceChunks) == 0 {
return
}
@ -24,7 +24,7 @@ func (fs *FilerSink) replicateChunks(ctx context.Context, sourceChunks []*filer_
wg.Add(1)
go func(chunk *filer_pb.FileChunk) {
defer wg.Done()
replicatedChunk, e := fs.replicateOneChunk(ctx, chunk, dir)
replicatedChunk, e := fs.replicateOneChunk(chunk, dir)
if e != nil {
err = e
}
@ -36,9 +36,9 @@ func (fs *FilerSink) replicateChunks(ctx context.Context, sourceChunks []*filer_
return
}
func (fs *FilerSink) replicateOneChunk(ctx context.Context, sourceChunk *filer_pb.FileChunk, dir string) (*filer_pb.FileChunk, error) {
func (fs *FilerSink) replicateOneChunk(sourceChunk *filer_pb.FileChunk, dir string) (*filer_pb.FileChunk, error) {
fileId, err := fs.fetchAndWrite(ctx, sourceChunk, dir)
fileId, err := fs.fetchAndWrite(sourceChunk, dir)
if err != nil {
return nil, fmt.Errorf("copy %s: %v", sourceChunk.GetFileIdString(), err)
}
@ -53,9 +53,9 @@ func (fs *FilerSink) replicateOneChunk(ctx context.Context, sourceChunk *filer_p
}, nil
}
func (fs *FilerSink) fetchAndWrite(ctx context.Context, sourceChunk *filer_pb.FileChunk, dir string) (fileId string, err error) {
func (fs *FilerSink) fetchAndWrite(sourceChunk *filer_pb.FileChunk, dir string) (fileId string, err error) {
filename, header, readCloser, err := fs.filerSource.ReadPart(ctx, sourceChunk.GetFileIdString())
filename, header, readCloser, err := fs.filerSource.ReadPart(sourceChunk.GetFileIdString())
if err != nil {
return "", fmt.Errorf("read part %s: %v", sourceChunk.GetFileIdString(), err)
}
@ -64,7 +64,7 @@ func (fs *FilerSink) fetchAndWrite(ctx context.Context, sourceChunk *filer_pb.Fi
var host string
var auth security.EncodedJwt
if err := fs.withFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
if err := fs.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.AssignVolumeRequest{
Count: 1,
@ -75,7 +75,7 @@ func (fs *FilerSink) fetchAndWrite(ctx context.Context, sourceChunk *filer_pb.Fi
ParentPath: dir,
}
resp, err := client.AssignVolume(ctx, request)
resp, err := client.AssignVolume(context.Background(), request)
if err != nil {
glog.V(0).Infof("assign volume failure %v: %v", request, err)
return err
@ -109,11 +109,11 @@ func (fs *FilerSink) fetchAndWrite(ctx context.Context, sourceChunk *filer_pb.Fi
return
}
func (fs *FilerSink) withFilerClient(ctx context.Context, fn func(context.Context, filer_pb.SeaweedFilerClient) error) error {
func (fs *FilerSink) withFilerClient(fn func(filer_pb.SeaweedFilerClient) error) error {
return util.WithCachedGrpcClient(ctx, func(ctx context.Context, grpcConnection *grpc.ClientConn) error {
return util.WithCachedGrpcClient(func(grpcConnection *grpc.ClientConn) error {
client := filer_pb.NewSeaweedFilerClient(grpcConnection)
return fn(ctx, client)
return fn(client)
}, fs.grpcAddress, fs.grpcDialOption)
}

View File

@ -64,8 +64,8 @@ func (fs *FilerSink) initialize(grpcAddress string, dir string,
return nil
}
func (fs *FilerSink) DeleteEntry(ctx context.Context, key string, isDirectory, deleteIncludeChunks bool) error {
return fs.withFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
func (fs *FilerSink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool) error {
return fs.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
dir, name := filer2.FullPath(key).DirAndName()
@ -76,7 +76,7 @@ func (fs *FilerSink) DeleteEntry(ctx context.Context, key string, isDirectory, d
}
glog.V(1).Infof("delete entry: %v", request)
_, err := client.DeleteEntry(ctx, request)
_, err := client.DeleteEntry(context.Background(), request)
if err != nil {
glog.V(0).Infof("delete entry %s: %v", key, err)
return fmt.Errorf("delete entry %s: %v", key, err)
@ -86,9 +86,9 @@ func (fs *FilerSink) DeleteEntry(ctx context.Context, key string, isDirectory, d
})
}
func (fs *FilerSink) CreateEntry(ctx context.Context, key string, entry *filer_pb.Entry) error {
func (fs *FilerSink) CreateEntry(key string, entry *filer_pb.Entry) error {
return fs.withFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
return fs.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
dir, name := filer2.FullPath(key).DirAndName()
@ -98,14 +98,14 @@ func (fs *FilerSink) CreateEntry(ctx context.Context, key string, entry *filer_p
Name: name,
}
glog.V(1).Infof("lookup: %v", lookupRequest)
if resp, err := client.LookupDirectoryEntry(ctx, lookupRequest); err == nil && resp.Entry != nil {
if resp, err := client.LookupDirectoryEntry(context.Background(), lookupRequest); err == nil && resp.Entry != nil {
if filer2.ETag(resp.Entry.Chunks) == filer2.ETag(entry.Chunks) {
glog.V(0).Infof("already replicated %s", key)
return nil
}
}
replicatedChunks, err := fs.replicateChunks(ctx, entry.Chunks, dir)
replicatedChunks, err := fs.replicateChunks(entry.Chunks, dir)
if err != nil {
glog.V(0).Infof("replicate entry chunks %s: %v", key, err)
@ -125,7 +125,7 @@ func (fs *FilerSink) CreateEntry(ctx context.Context, key string, entry *filer_p
}
glog.V(1).Infof("create: %v", request)
if err := filer_pb.CreateEntry(ctx, client, request); err != nil {
if err := filer_pb.CreateEntry(client, request); err != nil {
glog.V(0).Infof("create entry %s: %v", key, err)
return fmt.Errorf("create entry %s: %v", key, err)
}
@ -134,13 +134,13 @@ func (fs *FilerSink) CreateEntry(ctx context.Context, key string, entry *filer_p
})
}
func (fs *FilerSink) UpdateEntry(ctx context.Context, key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
func (fs *FilerSink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
dir, name := filer2.FullPath(key).DirAndName()
// read existing entry
var existingEntry *filer_pb.Entry
err = fs.withFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
err = fs.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.LookupDirectoryEntryRequest{
Directory: dir,
@ -148,7 +148,7 @@ func (fs *FilerSink) UpdateEntry(ctx context.Context, key string, oldEntry *file
}
glog.V(4).Infof("lookup entry: %v", request)
resp, err := client.LookupDirectoryEntry(ctx, request)
resp, err := client.LookupDirectoryEntry(context.Background(), request)
if err != nil {
glog.V(0).Infof("lookup %s: %v", key, err)
return err
@ -187,7 +187,7 @@ func (fs *FilerSink) UpdateEntry(ctx context.Context, key string, oldEntry *file
}
// replicate the chunks that are new in the source
replicatedChunks, err := fs.replicateChunks(ctx, newChunks, newParentPath)
replicatedChunks, err := fs.replicateChunks(newChunks, newParentPath)
if err != nil {
return true, fmt.Errorf("replicte %s chunks error: %v", key, err)
}
@ -195,14 +195,14 @@ func (fs *FilerSink) UpdateEntry(ctx context.Context, key string, oldEntry *file
}
// save updated meta data
return true, fs.withFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
return true, fs.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.UpdateEntryRequest{
Directory: newParentPath,
Entry: existingEntry,
}
if _, err := client.UpdateEntry(ctx, request); err != nil {
if _, err := client.UpdateEntry(context.Background(), request); err != nil {
return fmt.Errorf("update existingEntry %s: %v", key, err)
}

View File

@ -69,13 +69,13 @@ func (g *GcsSink) initialize(google_application_credentials, bucketName, dir str
return nil
}
func (g *GcsSink) DeleteEntry(ctx context.Context, key string, isDirectory, deleteIncludeChunks bool) error {
func (g *GcsSink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool) error {
if isDirectory {
key = key + "/"
}
if err := g.client.Bucket(g.bucket).Object(key).Delete(ctx); err != nil {
if err := g.client.Bucket(g.bucket).Object(key).Delete(context.Background()); err != nil {
return fmt.Errorf("gcs delete %s%s: %v", g.bucket, key, err)
}
@ -83,7 +83,7 @@ func (g *GcsSink) DeleteEntry(ctx context.Context, key string, isDirectory, dele
}
func (g *GcsSink) CreateEntry(ctx context.Context, key string, entry *filer_pb.Entry) error {
func (g *GcsSink) CreateEntry(key string, entry *filer_pb.Entry) error {
if entry.IsDirectory {
return nil
@ -92,11 +92,11 @@ func (g *GcsSink) CreateEntry(ctx context.Context, key string, entry *filer_pb.E
totalSize := filer2.TotalSize(entry.Chunks)
chunkViews := filer2.ViewFromChunks(entry.Chunks, 0, int(totalSize))
wc := g.client.Bucket(g.bucket).Object(key).NewWriter(ctx)
wc := g.client.Bucket(g.bucket).Object(key).NewWriter(context.Background())
for _, chunk := range chunkViews {
fileUrl, err := g.filerSource.LookupFileId(ctx, chunk.FileId)
fileUrl, err := g.filerSource.LookupFileId(chunk.FileId)
if err != nil {
return err
}
@ -119,7 +119,7 @@ func (g *GcsSink) CreateEntry(ctx context.Context, key string, entry *filer_pb.E
}
func (g *GcsSink) UpdateEntry(ctx context.Context, key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
func (g *GcsSink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
// TODO improve efficiency
return false, nil
}

View File

@ -1,7 +1,6 @@
package sink
import (
"context"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/replication/source"
"github.com/chrislusf/seaweedfs/weed/util"
@ -10,9 +9,9 @@ import (
type ReplicationSink interface {
GetName() string
Initialize(configuration util.Configuration, prefix string) error
DeleteEntry(ctx context.Context, key string, isDirectory, deleteIncludeChunks bool) error
CreateEntry(ctx context.Context, key string, entry *filer_pb.Entry) error
UpdateEntry(ctx context.Context, key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error)
DeleteEntry(key string, isDirectory, deleteIncludeChunks bool) error
CreateEntry(key string, entry *filer_pb.Entry) error
UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error)
GetSinkToDirectory() string
SetSourceFiler(s *source.FilerSource)
}

View File

@ -11,6 +11,7 @@ import (
"github.com/aws/aws-sdk-go/aws/session"
"github.com/aws/aws-sdk-go/service/s3"
"github.com/aws/aws-sdk-go/service/s3/s3iface"
"github.com/chrislusf/seaweedfs/weed/filer2"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
@ -77,7 +78,7 @@ func (s3sink *S3Sink) initialize(awsAccessKeyId, awsSecretAccessKey, region, buc
return nil
}
func (s3sink *S3Sink) DeleteEntry(ctx context.Context, key string, isDirectory, deleteIncludeChunks bool) error {
func (s3sink *S3Sink) DeleteEntry(key string, isDirectory, deleteIncludeChunks bool) error {
key = cleanKey(key)
@ -89,7 +90,7 @@ func (s3sink *S3Sink) DeleteEntry(ctx context.Context, key string, isDirectory,
}
func (s3sink *S3Sink) CreateEntry(ctx context.Context, key string, entry *filer_pb.Entry) error {
func (s3sink *S3Sink) CreateEntry(key string, entry *filer_pb.Entry) error {
key = cleanKey(key)
@ -112,7 +113,7 @@ func (s3sink *S3Sink) CreateEntry(ctx context.Context, key string, entry *filer_
wg.Add(1)
go func(chunk *filer2.ChunkView) {
defer wg.Done()
if part, uploadErr := s3sink.uploadPart(ctx, key, uploadId, partId, chunk); uploadErr != nil {
if part, uploadErr := s3sink.uploadPart(context.Background(), key, uploadId, partId, chunk); uploadErr != nil {
err = uploadErr
} else {
parts = append(parts, part)
@ -126,11 +127,11 @@ func (s3sink *S3Sink) CreateEntry(ctx context.Context, key string, entry *filer_
return err
}
return s3sink.completeMultipartUpload(ctx, key, uploadId, parts)
return s3sink.completeMultipartUpload(context.Background(), key, uploadId, parts)
}
func (s3sink *S3Sink) UpdateEntry(ctx context.Context, key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
func (s3sink *S3Sink) UpdateEntry(key string, oldEntry *filer_pb.Entry, newParentPath string, newEntry *filer_pb.Entry, deleteIncludeChunks bool) (foundExistingEntry bool, err error) {
key = cleanKey(key)
// TODO improve efficiency
return false, nil

View File

@ -157,7 +157,7 @@ func (s3sink *S3Sink) uploadPartCopy(key, uploadId string, partId int64, copySou
}
func (s3sink *S3Sink) buildReadSeeker(ctx context.Context, chunk *filer2.ChunkView) (io.ReadSeeker, error) {
fileUrl, err := s3sink.filerSource.LookupFileId(ctx, chunk.FileId)
fileUrl, err := s3sink.filerSource.LookupFileId(chunk.FileId)
if err != nil {
return nil, err
}

View File

@ -40,16 +40,16 @@ func (fs *FilerSource) initialize(grpcAddress string, dir string) (err error) {
return nil
}
func (fs *FilerSource) LookupFileId(ctx context.Context, part string) (fileUrl string, err error) {
func (fs *FilerSource) LookupFileId(part string) (fileUrl string, err error) {
vid2Locations := make(map[string]*filer_pb.Locations)
vid := volumeId(part)
err = fs.withFilerClient(ctx, fs.grpcDialOption, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
err = fs.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
glog.V(4).Infof("read lookup volume id locations: %v", vid)
resp, err := client.LookupVolume(ctx, &filer_pb.LookupVolumeRequest{
resp, err := client.LookupVolume(context.Background(), &filer_pb.LookupVolumeRequest{
VolumeIds: []string{vid},
})
if err != nil {
@ -78,9 +78,9 @@ func (fs *FilerSource) LookupFileId(ctx context.Context, part string) (fileUrl s
return
}
func (fs *FilerSource) ReadPart(ctx context.Context, part string) (filename string, header http.Header, readCloser io.ReadCloser, err error) {
func (fs *FilerSource) ReadPart(part string) (filename string, header http.Header, readCloser io.ReadCloser, err error) {
fileUrl, err := fs.LookupFileId(ctx, part)
fileUrl, err := fs.LookupFileId(part)
if err != nil {
return "", nil, nil, err
}
@ -90,11 +90,11 @@ func (fs *FilerSource) ReadPart(ctx context.Context, part string) (filename stri
return filename, header, readCloser, err
}
func (fs *FilerSource) withFilerClient(ctx context.Context, grpcDialOption grpc.DialOption, fn func(context.Context, filer_pb.SeaweedFilerClient) error) error {
func (fs *FilerSource) withFilerClient(fn func(filer_pb.SeaweedFilerClient) error) error {
return util.WithCachedGrpcClient(ctx, func(ctx2 context.Context, grpcConnection *grpc.ClientConn) error {
return util.WithCachedGrpcClient(func(grpcConnection *grpc.ClientConn) error {
client := filer_pb.NewSeaweedFilerClient(grpcConnection)
return fn(ctx2, client)
return fn(client)
}, fs.grpcAddress, fs.grpcDialOption)
}

View File

@ -13,7 +13,7 @@ import (
)
func (s3a *S3ApiServer) mkdir(ctx context.Context, parentDirectoryPath string, dirName string, fn func(entry *filer_pb.Entry)) error {
return s3a.withFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
return s3a.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
entry := &filer_pb.Entry{
Name: dirName,
@ -37,7 +37,7 @@ func (s3a *S3ApiServer) mkdir(ctx context.Context, parentDirectoryPath string, d
}
glog.V(1).Infof("mkdir: %v", request)
if err := filer_pb.CreateEntry(ctx, client, request); err != nil {
if err := filer_pb.CreateEntry(client, request); err != nil {
glog.V(0).Infof("mkdir %v: %v", request, err)
return fmt.Errorf("mkdir %s/%s: %v", parentDirectoryPath, dirName, err)
}
@ -47,7 +47,7 @@ func (s3a *S3ApiServer) mkdir(ctx context.Context, parentDirectoryPath string, d
}
func (s3a *S3ApiServer) mkFile(ctx context.Context, parentDirectoryPath string, fileName string, chunks []*filer_pb.FileChunk) error {
return s3a.withFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
return s3a.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
entry := &filer_pb.Entry{
Name: fileName,
@ -68,7 +68,7 @@ func (s3a *S3ApiServer) mkFile(ctx context.Context, parentDirectoryPath string,
}
glog.V(1).Infof("create file: %s/%s", parentDirectoryPath, fileName)
if err := filer_pb.CreateEntry(ctx, client, request); err != nil {
if err := filer_pb.CreateEntry(client, request); err != nil {
glog.V(0).Infof("create file %v:%v", request, err)
return fmt.Errorf("create file %s/%s: %v", parentDirectoryPath, fileName, err)
}
@ -79,7 +79,7 @@ func (s3a *S3ApiServer) mkFile(ctx context.Context, parentDirectoryPath string,
func (s3a *S3ApiServer) list(ctx context.Context, parentDirectoryPath, prefix, startFrom string, inclusive bool, limit int) (entries []*filer_pb.Entry, err error) {
err = s3a.withFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
err = s3a.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.ListEntriesRequest{
Directory: parentDirectoryPath,
@ -119,7 +119,7 @@ func (s3a *S3ApiServer) list(ctx context.Context, parentDirectoryPath, prefix, s
func (s3a *S3ApiServer) rm(ctx context.Context, parentDirectoryPath string, entryName string, isDirectory, isDeleteData, isRecursive bool) error {
return s3a.withFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
return s3a.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.DeleteEntryRequest{
Directory: parentDirectoryPath,
@ -143,7 +143,7 @@ func (s3a *S3ApiServer) streamRemove(ctx context.Context, quiet bool,
fn func() (finished bool, parentDirectoryPath string, entryName string, isDeleteData, isRecursive bool),
respFn func(err string)) error {
return s3a.withFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
return s3a.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
stream, err := client.StreamDeleteEntries(ctx)
if err != nil {
@ -196,7 +196,7 @@ func (s3a *S3ApiServer) streamRemove(ctx context.Context, quiet bool,
func (s3a *S3ApiServer) exists(ctx context.Context, parentDirectoryPath string, entryName string, isDirectory bool) (exists bool, err error) {
err = s3a.withFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
err = s3a.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.LookupDirectoryEntryRequest{
Directory: parentDirectoryPath,

View File

@ -80,7 +80,7 @@ func (s3a *S3ApiServer) DeleteBucketHandler(w http.ResponseWriter, r *http.Reque
bucket := vars["bucket"]
ctx := context.Background()
err := s3a.withFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
err := s3a.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
// delete collection
deleteCollectionRequest := &filer_pb.DeleteCollectionRequest{
@ -112,7 +112,7 @@ func (s3a *S3ApiServer) HeadBucketHandler(w http.ResponseWriter, r *http.Request
ctx := context.Background()
err := s3a.withFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
err := s3a.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.LookupDirectoryEntryRequest{
Directory: s3a.option.BucketsPath,

View File

@ -2,17 +2,18 @@ package s3api
import (
"bytes"
"context"
"encoding/base64"
"encoding/xml"
"fmt"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util"
"google.golang.org/grpc"
"net/http"
"net/url"
"time"
"google.golang.org/grpc"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util"
)
type mimeType string
@ -37,9 +38,9 @@ func encodeResponse(response interface{}) []byte {
return bytesBuffer.Bytes()
}
func (s3a *S3ApiServer) withFilerClient(ctx context.Context, fn func(filer_pb.SeaweedFilerClient) error) error {
func (s3a *S3ApiServer) withFilerClient(fn func(filer_pb.SeaweedFilerClient) error) error {
return util.WithCachedGrpcClient(ctx, func(ctx context.Context, grpcConnection *grpc.ClientConn) error {
return util.WithCachedGrpcClient(func(grpcConnection *grpc.ClientConn) error {
client := filer_pb.NewSeaweedFilerClient(grpcConnection)
return fn(client)
}, s3a.option.FilerGrpcAddress, s3a.option.GrpcDialOption)

View File

@ -95,7 +95,7 @@ func (s3a *S3ApiServer) listFilerEntries(ctx context.Context, bucket, originalPr
}
// check filer
err = s3a.withFilerClient(ctx, func(client filer_pb.SeaweedFilerClient) error {
err = s3a.withFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.ListEntriesRequest{
Directory: fmt.Sprintf("%s/%s/%s", s3a.option.BucketsPath, bucket, dir),

View File

@ -300,8 +300,8 @@ func (fs *FilerServer) AssignVolume(ctx context.Context, req *filer_pb.AssignVol
func (fs *FilerServer) DeleteCollection(ctx context.Context, req *filer_pb.DeleteCollectionRequest) (resp *filer_pb.DeleteCollectionResponse, err error) {
err = fs.filer.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
_, err := client.CollectionDelete(ctx, &master_pb.CollectionDeleteRequest{
err = fs.filer.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
_, err := client.CollectionDelete(context.Background(), &master_pb.CollectionDeleteRequest{
Name: req.GetCollection(),
})
return err

View File

@ -127,8 +127,8 @@ func maybeStartMetrics(fs *FilerServer, option *FilerOption) {
}
func readFilerConfiguration(grpcDialOption grpc.DialOption, masterGrpcAddress string) (metricsAddress string, metricsIntervalSec int, err error) {
err = operation.WithMasterServerClient(masterGrpcAddress, grpcDialOption, func(ctx context.Context, masterClient master_pb.SeaweedClient) error {
resp, err := masterClient.GetMasterConfiguration(ctx, &master_pb.GetMasterConfigurationRequest{})
err = operation.WithMasterServerClient(masterGrpcAddress, grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
resp, err := masterClient.GetMasterConfiguration(context.Background(), &master_pb.GetMasterConfigurationRequest{})
if err != nil {
return fmt.Errorf("get master %s configuration: %v", masterGrpcAddress, err)
}

View File

@ -4,6 +4,7 @@ import (
"context"
"github.com/chrislusf/raft"
"github.com/chrislusf/seaweedfs/weed/operation"
"github.com/chrislusf/seaweedfs/weed/pb/master_pb"
"github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
@ -57,8 +58,8 @@ func (ms *MasterServer) doDeleteNormalCollection(collectionName string) error {
}
for _, server := range collection.ListVolumeServers() {
err := operation.WithVolumeServerClient(server.Url(), ms.grpcDialOption, func(ctx context.Context, client volume_server_pb.VolumeServerClient) error {
_, deleteErr := client.DeleteCollection(ctx, &volume_server_pb.DeleteCollectionRequest{
err := operation.WithVolumeServerClient(server.Url(), ms.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
_, deleteErr := client.DeleteCollection(context.Background(), &volume_server_pb.DeleteCollectionRequest{
Collection: collectionName,
})
return deleteErr
@ -77,8 +78,8 @@ func (ms *MasterServer) doDeleteEcCollection(collectionName string) error {
listOfEcServers := ms.Topo.ListEcServersByCollection(collectionName)
for _, server := range listOfEcServers {
err := operation.WithVolumeServerClient(server, ms.grpcDialOption, func(ctx context.Context, client volume_server_pb.VolumeServerClient) error {
_, deleteErr := client.DeleteCollection(ctx, &volume_server_pb.DeleteCollectionRequest{
err := operation.WithVolumeServerClient(server, ms.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
_, deleteErr := client.DeleteCollection(context.Background(), &volume_server_pb.DeleteCollectionRequest{
Collection: collectionName,
})
return deleteErr

View File

@ -1,7 +1,6 @@
package weed_server
import (
"context"
"fmt"
"net/http"
"net/http/httputil"
@ -89,7 +88,7 @@ func NewMasterServer(r *mux.Router, option *MasterOption, peers []string) *Maste
preallocateSize: preallocateSize,
clientChans: make(map[string]chan *master_pb.VolumeLocation),
grpcDialOption: grpcDialOption,
MasterClient: wdclient.NewMasterClient(context.Background(), grpcDialOption, "master", peers),
MasterClient: wdclient.NewMasterClient(grpcDialOption, "master", peers),
}
ms.bounedLeaderChan = make(chan int, 16)

View File

@ -25,8 +25,8 @@ func (ms *MasterServer) collectionDeleteHandler(w http.ResponseWriter, r *http.R
return
}
for _, server := range collection.ListVolumeServers() {
err := operation.WithVolumeServerClient(server.Url(), ms.grpcDialOption, func(ctx context.Context, client volume_server_pb.VolumeServerClient) error {
_, deleteErr := client.DeleteCollection(ctx, &volume_server_pb.DeleteCollectionRequest{
err := operation.WithVolumeServerClient(server.Url(), ms.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
_, deleteErr := client.DeleteCollection(context.Background(), &volume_server_pb.DeleteCollectionRequest{
Collection: collection.Name,
})
return deleteErr

View File

@ -41,9 +41,9 @@ func (vs *VolumeServer) VolumeCopy(ctx context.Context, req *volume_server_pb.Vo
// confirm size and timestamp
var volFileInfoResp *volume_server_pb.ReadVolumeFileStatusResponse
var volumeFileName, idxFileName, datFileName string
err := operation.WithVolumeServerClient(req.SourceDataNode, vs.grpcDialOption, func(ctx context.Context, client volume_server_pb.VolumeServerClient) error {
err := operation.WithVolumeServerClient(req.SourceDataNode, vs.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
var err error
volFileInfoResp, err = client.ReadVolumeFileStatus(ctx,
volFileInfoResp, err = client.ReadVolumeFileStatus(context.Background(),
&volume_server_pb.ReadVolumeFileStatusRequest{
VolumeId: req.VolumeId,
})

View File

@ -106,7 +106,7 @@ func (vs *VolumeServer) VolumeEcShardsCopy(ctx context.Context, req *volume_serv
baseFileName := storage.VolumeFileName(location.Directory, req.Collection, int(req.VolumeId))
err := operation.WithVolumeServerClient(req.SourceDataNode, vs.grpcDialOption, func(ctx context.Context, client volume_server_pb.VolumeServerClient) error {
err := operation.WithVolumeServerClient(req.SourceDataNode, vs.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
// copy ec data slices
for _, shardId := range req.ShardIds {

View File

@ -1,7 +1,6 @@
package weed_server
import (
"context"
"encoding/json"
"net/http"
"strings"
@ -45,7 +44,7 @@ func (vs *VolumeServer) FileGet(req *volume_server_pb.FileGetRequest, stream vol
if hasVolume {
count, err = vs.store.ReadVolumeNeedle(volumeId, n)
} else if hasEcVolume {
count, err = vs.store.ReadEcShardNeedle(context.Background(), volumeId, n)
count, err = vs.store.ReadEcShardNeedle(volumeId, n)
}
if err != nil || count < 0 {

View File

@ -2,7 +2,7 @@ package weed_server
import (
"bytes"
"context"
"encoding/json"
"errors"
"fmt"
"io"
@ -15,8 +15,6 @@ import (
"strings"
"time"
"encoding/json"
"github.com/chrislusf/seaweedfs/weed/glog"
"github.com/chrislusf/seaweedfs/weed/images"
"github.com/chrislusf/seaweedfs/weed/operation"
@ -86,7 +84,7 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
if hasVolume {
count, err = vs.store.ReadVolumeNeedle(volumeId, n)
} else if hasEcVolume {
count, err = vs.store.ReadEcShardNeedle(context.Background(), volumeId, n)
count, err = vs.store.ReadEcShardNeedle(volumeId, n)
}
// glog.V(4).Infoln("read bytes", count, "error", err)
if err != nil || count < 0 {

View File

@ -96,11 +96,11 @@ func NewWebDavFileSystem(option *WebDavOption) (webdav.FileSystem, error) {
}, nil
}
func (fs *WebDavFileSystem) WithFilerClient(ctx context.Context, fn func(context.Context, filer_pb.SeaweedFilerClient) error) error {
func (fs *WebDavFileSystem) WithFilerClient(fn func(filer_pb.SeaweedFilerClient) error) error {
return util.WithCachedGrpcClient(ctx, func(ctx2 context.Context, grpcConnection *grpc.ClientConn) error {
return util.WithCachedGrpcClient(func(grpcConnection *grpc.ClientConn) error {
client := filer_pb.NewSeaweedFilerClient(grpcConnection)
return fn(ctx2, client)
return fn(client)
}, fs.option.FilerGrpcAddress, fs.option.GrpcDialOption)
}
@ -135,7 +135,7 @@ func (fs *WebDavFileSystem) Mkdir(ctx context.Context, fullDirPath string, perm
return os.ErrExist
}
return fs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
return fs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
dir, name := filer2.FullPath(fullDirPath).DirAndName()
request := &filer_pb.CreateEntryRequest{
Directory: dir,
@ -153,7 +153,7 @@ func (fs *WebDavFileSystem) Mkdir(ctx context.Context, fullDirPath string, perm
}
glog.V(1).Infof("mkdir: %v", request)
if err := filer_pb.CreateEntry(ctx, client, request); err != nil {
if err := filer_pb.CreateEntry(client, request); err != nil {
return fmt.Errorf("mkdir %s/%s: %v", dir, name, err)
}
@ -184,8 +184,8 @@ func (fs *WebDavFileSystem) OpenFile(ctx context.Context, fullFilePath string, f
}
dir, name := filer2.FullPath(fullFilePath).DirAndName()
err = fs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
if err := filer_pb.CreateEntry(ctx, client, &filer_pb.CreateEntryRequest{
err = fs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
if err := filer_pb.CreateEntry(client, &filer_pb.CreateEntryRequest{
Directory: dir,
Entry: &filer_pb.Entry{
Name: name,
@ -249,7 +249,7 @@ func (fs *WebDavFileSystem) removeAll(ctx context.Context, fullFilePath string)
//_, err = fs.db.Exec(`delete from filesystem where fullFilePath = ?`, fullFilePath)
}
dir, name := filer2.FullPath(fullFilePath).DirAndName()
err = fs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
err = fs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.DeleteEntryRequest{
Directory: dir,
@ -308,7 +308,7 @@ func (fs *WebDavFileSystem) Rename(ctx context.Context, oldName, newName string)
oldDir, oldBaseName := filer2.FullPath(oldName).DirAndName()
newDir, newBaseName := filer2.FullPath(newName).DirAndName()
return fs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
return fs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.AtomicRenameEntryRequest{
OldDirectory: oldDir,
@ -336,7 +336,7 @@ func (fs *WebDavFileSystem) stat(ctx context.Context, fullFilePath string) (os.F
fullpath := filer2.FullPath(fullFilePath)
var fi FileInfo
entry, err := filer2.GetEntry(ctx, fs, fullpath)
entry, err := filer2.GetEntry(fs, fullpath)
if entry == nil {
return nil, os.ErrNotExist
}
@ -372,7 +372,7 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
var err error
ctx := context.Background()
if f.entry == nil {
f.entry, err = filer2.GetEntry(ctx, f.fs, filer2.FullPath(f.name))
f.entry, err = filer2.GetEntry(f.fs, filer2.FullPath(f.name))
}
if f.entry == nil {
@ -386,7 +386,7 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
var auth security.EncodedJwt
var collection, replication string
if err = f.fs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
if err = f.fs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.AssignVolumeRequest{
Count: 1,
@ -434,7 +434,7 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
f.entry.Chunks = append(f.entry.Chunks, chunk)
err = f.fs.WithFilerClient(ctx, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
err = f.fs.WithFilerClient(func(client filer_pb.SeaweedFilerClient) error {
f.entry.Attributes.Mtime = time.Now().Unix()
f.entry.Attributes.Collection = collection
f.entry.Attributes.Replication = replication
@ -474,10 +474,9 @@ func (f *WebDavFile) Close() error {
func (f *WebDavFile) Read(p []byte) (readSize int, err error) {
glog.V(2).Infof("WebDavFileSystem.Read %v", f.name)
ctx := context.Background()
if f.entry == nil {
f.entry, err = filer2.GetEntry(ctx, f.fs, filer2.FullPath(f.name))
f.entry, err = filer2.GetEntry(f.fs, filer2.FullPath(f.name))
}
if f.entry == nil {
return 0, err
@ -493,7 +492,7 @@ func (f *WebDavFile) Read(p []byte) (readSize int, err error) {
}
chunkViews := filer2.ViewFromVisibleIntervals(f.entryViewCache, f.off, len(p))
totalRead, err := filer2.ReadIntoBuffer(ctx, f.fs, filer2.FullPath(f.name), p, chunkViews, f.off)
totalRead, err := filer2.ReadIntoBuffer(f.fs, filer2.FullPath(f.name), p, chunkViews, f.off)
if err != nil {
return 0, err
}
@ -512,11 +511,10 @@ func (f *WebDavFile) Read(p []byte) (readSize int, err error) {
func (f *WebDavFile) Readdir(count int) (ret []os.FileInfo, err error) {
glog.V(2).Infof("WebDavFileSystem.Readdir %v count %d", f.name, count)
ctx := context.Background()
dir, _ := filer2.FullPath(f.name).DirAndName()
err = filer2.ReadDirAllEntries(ctx, f.fs, filer2.FullPath(dir), "", func(entry *filer_pb.Entry, isLast bool) {
err = filer2.ReadDirAllEntries(f.fs, filer2.FullPath(dir), "", func(entry *filer_pb.Entry, isLast bool) {
fi := FileInfo{
size: int64(filer2.TotalSize(entry.GetChunks())),
name: entry.Name,

View File

@ -48,11 +48,9 @@ func (c *commandBucketCreate) Do(args []string, commandEnv *CommandEnv, writer i
return parseErr
}
ctx := context.Background()
err = commandEnv.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
err = commandEnv.withFilerClient(ctx, filerServer, filerPort, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
resp, err := client.GetFilerConfiguration(ctx, &filer_pb.GetFilerConfigurationRequest{})
resp, err := client.GetFilerConfiguration(context.Background(), &filer_pb.GetFilerConfigurationRequest{})
if err != nil {
return fmt.Errorf("get filer %s:%d configuration: %v", filerServer, filerPort, err)
}
@ -72,7 +70,7 @@ func (c *commandBucketCreate) Do(args []string, commandEnv *CommandEnv, writer i
},
}
if err := filer_pb.CreateEntry(ctx, client, &filer_pb.CreateEntryRequest{
if err := filer_pb.CreateEntry(client, &filer_pb.CreateEntryRequest{
Directory: filerBucketsPath,
Entry: entry,
}); err != nil {

View File

@ -44,17 +44,15 @@ func (c *commandBucketDelete) Do(args []string, commandEnv *CommandEnv, writer i
return parseErr
}
ctx := context.Background()
err = commandEnv.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
err = commandEnv.withFilerClient(ctx, filerServer, filerPort, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
resp, err := client.GetFilerConfiguration(ctx, &filer_pb.GetFilerConfigurationRequest{})
resp, err := client.GetFilerConfiguration(context.Background(), &filer_pb.GetFilerConfigurationRequest{})
if err != nil {
return fmt.Errorf("get filer %s:%d configuration: %v", filerServer, filerPort, err)
}
filerBucketsPath := resp.DirBuckets
if _, err := client.DeleteEntry(ctx, &filer_pb.DeleteEntryRequest{
if _, err := client.DeleteEntry(context.Background(), &filer_pb.DeleteEntryRequest{
Directory: filerBucketsPath,
Name: *bucketName,
IsDeleteData: false,

View File

@ -39,17 +39,15 @@ func (c *commandBucketList) Do(args []string, commandEnv *CommandEnv, writer io.
return parseErr
}
ctx := context.Background()
err = commandEnv.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
err = commandEnv.withFilerClient(ctx, filerServer, filerPort, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
resp, err := client.GetFilerConfiguration(ctx, &filer_pb.GetFilerConfigurationRequest{})
resp, err := client.GetFilerConfiguration(context.Background(), &filer_pb.GetFilerConfigurationRequest{})
if err != nil {
return fmt.Errorf("get filer %s:%d configuration: %v", filerServer, filerPort, err)
}
filerBucketsPath := resp.DirBuckets
stream, err := client.ListEntries(ctx, &filer_pb.ListEntriesRequest{
stream, err := client.ListEntries(context.Background(), &filer_pb.ListEntriesRequest{
Directory: filerBucketsPath,
Limit: math.MaxUint32,
})

View File

@ -34,9 +34,8 @@ func (c *commandCollectionDelete) Do(args []string, commandEnv *CommandEnv, writ
collectionName := args[0]
ctx := context.Background()
err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
_, err = client.CollectionDelete(ctx, &master_pb.CollectionDeleteRequest{
err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
_, err = client.CollectionDelete(context.Background(), &master_pb.CollectionDeleteRequest{
Name: collectionName,
})
return err

View File

@ -41,9 +41,8 @@ func (c *commandCollectionList) Do(args []string, commandEnv *CommandEnv, writer
func ListCollectionNames(commandEnv *CommandEnv, includeNormalVolumes, includeEcVolumes bool) (collections []string, err error) {
var resp *master_pb.CollectionListResponse
ctx := context.Background()
err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
resp, err = client.CollectionList(ctx, &master_pb.CollectionListRequest{
err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
resp, err = client.CollectionList(context.Background(), &master_pb.CollectionListRequest{
IncludeNormalVolumes: includeNormalVolumes,
IncludeEcVolumes: includeEcVolumes,
})

View File

@ -107,10 +107,8 @@ func (c *commandEcBalance) Do(args []string, commandEnv *CommandEnv, writer io.W
return nil
}
ctx := context.Background()
// collect all ec nodes
allEcNodes, totalFreeEcSlots, err := collectEcNodes(ctx, commandEnv, *dc)
allEcNodes, totalFreeEcSlots, err := collectEcNodes(commandEnv, *dc)
if err != nil {
return err
}
@ -138,7 +136,7 @@ func (c *commandEcBalance) Do(args []string, commandEnv *CommandEnv, writer io.W
}
}
if err := balanceEcRacks(ctx, commandEnv, racks, *applyBalancing); err != nil {
if err := balanceEcRacks(commandEnv, racks, *applyBalancing); err != nil {
return fmt.Errorf("balance ec racks: %v", err)
}
@ -170,11 +168,11 @@ func balanceEcVolumes(commandEnv *CommandEnv, collection string, allEcNodes []*E
return fmt.Errorf("delete duplicated collection %s ec shards: %v", collection, err)
}
if err := balanceEcShardsAcrossRacks(ctx, commandEnv, allEcNodes, racks, collection, applyBalancing); err != nil {
if err := balanceEcShardsAcrossRacks(commandEnv, allEcNodes, racks, collection, applyBalancing); err != nil {
return fmt.Errorf("balance across racks collection %s ec shards: %v", collection, err)
}
if err := balanceEcShardsWithinRacks(ctx, commandEnv, allEcNodes, racks, collection, applyBalancing); err != nil {
if err := balanceEcShardsWithinRacks(commandEnv, allEcNodes, racks, collection, applyBalancing); err != nil {
return fmt.Errorf("balance across racks collection %s ec shards: %v", collection, err)
}
@ -186,14 +184,14 @@ func deleteDuplicatedEcShards(ctx context.Context, commandEnv *CommandEnv, allEc
vidLocations := collectVolumeIdToEcNodes(allEcNodes)
// deduplicate ec shards
for vid, locations := range vidLocations {
if err := doDeduplicateEcShards(ctx, commandEnv, collection, vid, locations, applyBalancing); err != nil {
if err := doDeduplicateEcShards(commandEnv, collection, vid, locations, applyBalancing); err != nil {
return err
}
}
return nil
}
func doDeduplicateEcShards(ctx context.Context, commandEnv *CommandEnv, collection string, vid needle.VolumeId, locations []*EcNode, applyBalancing bool) error {
func doDeduplicateEcShards(commandEnv *CommandEnv, collection string, vid needle.VolumeId, locations []*EcNode, applyBalancing bool) error {
// check whether this volume has ecNodes that are over average
shardToLocations := make([][]*EcNode, erasure_coding.TotalShardsCount)
@ -215,10 +213,10 @@ func doDeduplicateEcShards(ctx context.Context, commandEnv *CommandEnv, collecti
duplicatedShardIds := []uint32{uint32(shardId)}
for _, ecNode := range ecNodes[1:] {
if err := unmountEcShards(ctx, commandEnv.option.GrpcDialOption, vid, ecNode.info.Id, duplicatedShardIds); err != nil {
if err := unmountEcShards(commandEnv.option.GrpcDialOption, vid, ecNode.info.Id, duplicatedShardIds); err != nil {
return err
}
if err := sourceServerDeleteEcShards(ctx, commandEnv.option.GrpcDialOption, collection, vid, ecNode.info.Id, duplicatedShardIds); err != nil {
if err := sourceServerDeleteEcShards(commandEnv.option.GrpcDialOption, collection, vid, ecNode.info.Id, duplicatedShardIds); err != nil {
return err
}
ecNode.deleteEcVolumeShards(vid, duplicatedShardIds)
@ -227,19 +225,19 @@ func doDeduplicateEcShards(ctx context.Context, commandEnv *CommandEnv, collecti
return nil
}
func balanceEcShardsAcrossRacks(ctx context.Context, commandEnv *CommandEnv, allEcNodes []*EcNode, racks map[RackId]*EcRack, collection string, applyBalancing bool) error {
func balanceEcShardsAcrossRacks(commandEnv *CommandEnv, allEcNodes []*EcNode, racks map[RackId]*EcRack, collection string, applyBalancing bool) error {
// collect vid => []ecNode, since previous steps can change the locations
vidLocations := collectVolumeIdToEcNodes(allEcNodes)
// spread the ec shards evenly
for vid, locations := range vidLocations {
if err := doBalanceEcShardsAcrossRacks(ctx, commandEnv, collection, vid, locations, racks, applyBalancing); err != nil {
if err := doBalanceEcShardsAcrossRacks(commandEnv, collection, vid, locations, racks, applyBalancing); err != nil {
return err
}
}
return nil
}
func doBalanceEcShardsAcrossRacks(ctx context.Context, commandEnv *CommandEnv, collection string, vid needle.VolumeId, locations []*EcNode, racks map[RackId]*EcRack, applyBalancing bool) error {
func doBalanceEcShardsAcrossRacks(commandEnv *CommandEnv, collection string, vid needle.VolumeId, locations []*EcNode, racks map[RackId]*EcRack, applyBalancing bool) error {
// calculate average number of shards an ec rack should have for one volume
averageShardsPerEcRack := ceilDivide(erasure_coding.TotalShardsCount, len(racks))
@ -274,7 +272,7 @@ func doBalanceEcShardsAcrossRacks(ctx context.Context, commandEnv *CommandEnv, c
for _, n := range racks[rackId].ecNodes {
possibleDestinationEcNodes = append(possibleDestinationEcNodes, n)
}
err := pickOneEcNodeAndMoveOneShard(ctx, commandEnv, averageShardsPerEcRack, ecNode, collection, vid, shardId, possibleDestinationEcNodes, applyBalancing)
err := pickOneEcNodeAndMoveOneShard(commandEnv, averageShardsPerEcRack, ecNode, collection, vid, shardId, possibleDestinationEcNodes, applyBalancing)
if err != nil {
return err
}
@ -306,7 +304,7 @@ func pickOneRack(rackToEcNodes map[RackId]*EcRack, rackToShardCount map[string]i
return ""
}
func balanceEcShardsWithinRacks(ctx context.Context, commandEnv *CommandEnv, allEcNodes []*EcNode, racks map[RackId]*EcRack, collection string, applyBalancing bool) error {
func balanceEcShardsWithinRacks(commandEnv *CommandEnv, allEcNodes []*EcNode, racks map[RackId]*EcRack, collection string, applyBalancing bool) error {
// collect vid => []ecNode, since previous steps can change the locations
vidLocations := collectVolumeIdToEcNodes(allEcNodes)
@ -330,7 +328,7 @@ func balanceEcShardsWithinRacks(ctx context.Context, commandEnv *CommandEnv, all
}
sourceEcNodes := rackEcNodesWithVid[rackId]
averageShardsPerEcNode := ceilDivide(rackToShardCount[rackId], len(possibleDestinationEcNodes))
if err := doBalanceEcShardsWithinOneRack(ctx, commandEnv, averageShardsPerEcNode, collection, vid, sourceEcNodes, possibleDestinationEcNodes, applyBalancing); err != nil {
if err := doBalanceEcShardsWithinOneRack(commandEnv, averageShardsPerEcNode, collection, vid, sourceEcNodes, possibleDestinationEcNodes, applyBalancing); err != nil {
return err
}
}
@ -338,7 +336,7 @@ func balanceEcShardsWithinRacks(ctx context.Context, commandEnv *CommandEnv, all
return nil
}
func doBalanceEcShardsWithinOneRack(ctx context.Context, commandEnv *CommandEnv, averageShardsPerEcNode int, collection string, vid needle.VolumeId, existingLocations, possibleDestinationEcNodes []*EcNode, applyBalancing bool) error {
func doBalanceEcShardsWithinOneRack(commandEnv *CommandEnv, averageShardsPerEcNode int, collection string, vid needle.VolumeId, existingLocations, possibleDestinationEcNodes []*EcNode, applyBalancing bool) error {
for _, ecNode := range existingLocations {
@ -353,7 +351,7 @@ func doBalanceEcShardsWithinOneRack(ctx context.Context, commandEnv *CommandEnv,
fmt.Printf("%s has %d overlimit, moving ec shard %d.%d\n", ecNode.info.Id, overLimitCount, vid, shardId)
err := pickOneEcNodeAndMoveOneShard(ctx, commandEnv, averageShardsPerEcNode, ecNode, collection, vid, shardId, possibleDestinationEcNodes, applyBalancing)
err := pickOneEcNodeAndMoveOneShard(commandEnv, averageShardsPerEcNode, ecNode, collection, vid, shardId, possibleDestinationEcNodes, applyBalancing)
if err != nil {
return err
}
@ -365,18 +363,18 @@ func doBalanceEcShardsWithinOneRack(ctx context.Context, commandEnv *CommandEnv,
return nil
}
func balanceEcRacks(ctx context.Context, commandEnv *CommandEnv, racks map[RackId]*EcRack, applyBalancing bool) error {
func balanceEcRacks(commandEnv *CommandEnv, racks map[RackId]*EcRack, applyBalancing bool) error {
// balance one rack for all ec shards
for _, ecRack := range racks {
if err := doBalanceEcRack(ctx, commandEnv, ecRack, applyBalancing); err != nil {
if err := doBalanceEcRack(commandEnv, ecRack, applyBalancing); err != nil {
return err
}
}
return nil
}
func doBalanceEcRack(ctx context.Context, commandEnv *CommandEnv, ecRack *EcRack, applyBalancing bool) error {
func doBalanceEcRack(commandEnv *CommandEnv, ecRack *EcRack, applyBalancing bool) error {
if len(ecRack.ecNodes) <= 1 {
return nil
@ -421,7 +419,7 @@ func doBalanceEcRack(ctx context.Context, commandEnv *CommandEnv, ecRack *EcRack
fmt.Printf("%s moves ec shards %d.%d to %s\n", fullNode.info.Id, shards.Id, shardId, emptyNode.info.Id)
err := moveMountedShardToEcNode(ctx, commandEnv, fullNode, shards.Collection, needle.VolumeId(shards.Id), shardId, emptyNode, applyBalancing)
err := moveMountedShardToEcNode(commandEnv, fullNode, shards.Collection, needle.VolumeId(shards.Id), shardId, emptyNode, applyBalancing)
if err != nil {
return err
}
@ -440,7 +438,7 @@ func doBalanceEcRack(ctx context.Context, commandEnv *CommandEnv, ecRack *EcRack
return nil
}
func pickOneEcNodeAndMoveOneShard(ctx context.Context, commandEnv *CommandEnv, averageShardsPerEcNode int, existingLocation *EcNode, collection string, vid needle.VolumeId, shardId erasure_coding.ShardId, possibleDestinationEcNodes []*EcNode, applyBalancing bool) error {
func pickOneEcNodeAndMoveOneShard(commandEnv *CommandEnv, averageShardsPerEcNode int, existingLocation *EcNode, collection string, vid needle.VolumeId, shardId erasure_coding.ShardId, possibleDestinationEcNodes []*EcNode, applyBalancing bool) error {
sortEcNodesByFreeslotsDecending(possibleDestinationEcNodes)
@ -458,7 +456,7 @@ func pickOneEcNodeAndMoveOneShard(ctx context.Context, commandEnv *CommandEnv, a
fmt.Printf("%s moves ec shard %d.%d to %s\n", existingLocation.info.Id, vid, shardId, destEcNode.info.Id)
err := moveMountedShardToEcNode(ctx, commandEnv, existingLocation, collection, vid, shardId, destEcNode, applyBalancing)
err := moveMountedShardToEcNode(commandEnv, existingLocation, collection, vid, shardId, destEcNode, applyBalancing)
if err != nil {
return err
}

View File

@ -15,26 +15,26 @@ import (
"google.golang.org/grpc"
)
func moveMountedShardToEcNode(ctx context.Context, commandEnv *CommandEnv, existingLocation *EcNode, collection string, vid needle.VolumeId, shardId erasure_coding.ShardId, destinationEcNode *EcNode, applyBalancing bool) (err error) {
func moveMountedShardToEcNode(commandEnv *CommandEnv, existingLocation *EcNode, collection string, vid needle.VolumeId, shardId erasure_coding.ShardId, destinationEcNode *EcNode, applyBalancing bool) (err error) {
copiedShardIds := []uint32{uint32(shardId)}
if applyBalancing {
// ask destination node to copy shard and the ecx file from source node, and mount it
copiedShardIds, err = oneServerCopyAndMountEcShardsFromSource(ctx, commandEnv.option.GrpcDialOption, destinationEcNode, []uint32{uint32(shardId)}, vid, collection, existingLocation.info.Id)
copiedShardIds, err = oneServerCopyAndMountEcShardsFromSource(commandEnv.option.GrpcDialOption, destinationEcNode, []uint32{uint32(shardId)}, vid, collection, existingLocation.info.Id)
if err != nil {
return err
}
// unmount the to be deleted shards
err = unmountEcShards(ctx, commandEnv.option.GrpcDialOption, vid, existingLocation.info.Id, copiedShardIds)
err = unmountEcShards(commandEnv.option.GrpcDialOption, vid, existingLocation.info.Id, copiedShardIds)
if err != nil {
return err
}
// ask source node to delete the shard, and maybe the ecx file
err = sourceServerDeleteEcShards(ctx, commandEnv.option.GrpcDialOption, collection, vid, existingLocation.info.Id, copiedShardIds)
err = sourceServerDeleteEcShards(commandEnv.option.GrpcDialOption, collection, vid, existingLocation.info.Id, copiedShardIds)
if err != nil {
return err
}
@ -50,18 +50,18 @@ func moveMountedShardToEcNode(ctx context.Context, commandEnv *CommandEnv, exist
}
func oneServerCopyAndMountEcShardsFromSource(ctx context.Context, grpcDialOption grpc.DialOption,
func oneServerCopyAndMountEcShardsFromSource(grpcDialOption grpc.DialOption,
targetServer *EcNode, shardIdsToCopy []uint32,
volumeId needle.VolumeId, collection string, existingLocation string) (copiedShardIds []uint32, err error) {
fmt.Printf("allocate %d.%v %s => %s\n", volumeId, shardIdsToCopy, existingLocation, targetServer.info.Id)
err = operation.WithVolumeServerClient(targetServer.info.Id, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
err = operation.WithVolumeServerClient(targetServer.info.Id, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
if targetServer.info.Id != existingLocation {
fmt.Printf("copy %d.%v %s => %s\n", volumeId, shardIdsToCopy, existingLocation, targetServer.info.Id)
_, copyErr := volumeServerClient.VolumeEcShardsCopy(ctx, &volume_server_pb.VolumeEcShardsCopyRequest{
_, copyErr := volumeServerClient.VolumeEcShardsCopy(context.Background(), &volume_server_pb.VolumeEcShardsCopyRequest{
VolumeId: uint32(volumeId),
Collection: collection,
ShardIds: shardIdsToCopy,
@ -76,7 +76,7 @@ func oneServerCopyAndMountEcShardsFromSource(ctx context.Context, grpcDialOption
}
fmt.Printf("mount %d.%v on %s\n", volumeId, shardIdsToCopy, targetServer.info.Id)
_, mountErr := volumeServerClient.VolumeEcShardsMount(ctx, &volume_server_pb.VolumeEcShardsMountRequest{
_, mountErr := volumeServerClient.VolumeEcShardsMount(context.Background(), &volume_server_pb.VolumeEcShardsMountRequest{
VolumeId: uint32(volumeId),
Collection: collection,
ShardIds: shardIdsToCopy,
@ -178,12 +178,12 @@ type EcRack struct {
freeEcSlot int
}
func collectEcNodes(ctx context.Context, commandEnv *CommandEnv, selectedDataCenter string) (ecNodes []*EcNode, totalFreeEcSlots int, err error) {
func collectEcNodes(commandEnv *CommandEnv, selectedDataCenter string) (ecNodes []*EcNode, totalFreeEcSlots int, err error) {
// list all possible locations
var resp *master_pb.VolumeListResponse
err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
resp, err = client.VolumeList(ctx, &master_pb.VolumeListRequest{})
err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
return err
})
if err != nil {
@ -211,13 +211,12 @@ func collectEcNodes(ctx context.Context, commandEnv *CommandEnv, selectedDataCen
return
}
func sourceServerDeleteEcShards(ctx context.Context, grpcDialOption grpc.DialOption,
collection string, volumeId needle.VolumeId, sourceLocation string, toBeDeletedShardIds []uint32) error {
func sourceServerDeleteEcShards(grpcDialOption grpc.DialOption, collection string, volumeId needle.VolumeId, sourceLocation string, toBeDeletedShardIds []uint32) error {
fmt.Printf("delete %d.%v from %s\n", volumeId, toBeDeletedShardIds, sourceLocation)
return operation.WithVolumeServerClient(sourceLocation, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, deleteErr := volumeServerClient.VolumeEcShardsDelete(ctx, &volume_server_pb.VolumeEcShardsDeleteRequest{
return operation.WithVolumeServerClient(sourceLocation, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, deleteErr := volumeServerClient.VolumeEcShardsDelete(context.Background(), &volume_server_pb.VolumeEcShardsDeleteRequest{
VolumeId: uint32(volumeId),
Collection: collection,
ShardIds: toBeDeletedShardIds,
@ -227,13 +226,12 @@ func sourceServerDeleteEcShards(ctx context.Context, grpcDialOption grpc.DialOpt
}
func unmountEcShards(ctx context.Context, grpcDialOption grpc.DialOption,
volumeId needle.VolumeId, sourceLocation string, toBeUnmountedhardIds []uint32) error {
func unmountEcShards(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceLocation string, toBeUnmountedhardIds []uint32) error {
fmt.Printf("unmount %d.%v from %s\n", volumeId, toBeUnmountedhardIds, sourceLocation)
return operation.WithVolumeServerClient(sourceLocation, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, deleteErr := volumeServerClient.VolumeEcShardsUnmount(ctx, &volume_server_pb.VolumeEcShardsUnmountRequest{
return operation.WithVolumeServerClient(sourceLocation, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, deleteErr := volumeServerClient.VolumeEcShardsUnmount(context.Background(), &volume_server_pb.VolumeEcShardsUnmountRequest{
VolumeId: uint32(volumeId),
ShardIds: toBeUnmountedhardIds,
})
@ -241,13 +239,12 @@ func unmountEcShards(ctx context.Context, grpcDialOption grpc.DialOption,
})
}
func mountEcShards(ctx context.Context, grpcDialOption grpc.DialOption,
collection string, volumeId needle.VolumeId, sourceLocation string, toBeMountedhardIds []uint32) error {
func mountEcShards(grpcDialOption grpc.DialOption, collection string, volumeId needle.VolumeId, sourceLocation string, toBeMountedhardIds []uint32) error {
fmt.Printf("mount %d.%v on %s\n", volumeId, toBeMountedhardIds, sourceLocation)
return operation.WithVolumeServerClient(sourceLocation, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, mountErr := volumeServerClient.VolumeEcShardsMount(ctx, &volume_server_pb.VolumeEcShardsMountRequest{
return operation.WithVolumeServerClient(sourceLocation, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, mountErr := volumeServerClient.VolumeEcShardsMount(context.Background(), &volume_server_pb.VolumeEcShardsMountRequest{
VolumeId: uint32(volumeId),
Collection: collection,
ShardIds: toBeMountedhardIds,

View File

@ -43,25 +43,24 @@ func (c *commandEcDecode) Do(args []string, commandEnv *CommandEnv, writer io.Wr
return nil
}
ctx := context.Background()
vid := needle.VolumeId(*volumeId)
// collect topology information
topologyInfo, err := collectTopologyInfo(ctx, commandEnv)
topologyInfo, err := collectTopologyInfo(commandEnv)
if err != nil {
return err
}
// volumeId is provided
if vid != 0 {
return doEcDecode(ctx, commandEnv, topologyInfo, *collection, vid)
return doEcDecode(commandEnv, topologyInfo, *collection, vid)
}
// apply to all volumes in the collection
volumeIds := collectEcShardIds(topologyInfo, *collection)
fmt.Printf("ec encode volumes: %v\n", volumeIds)
for _, vid := range volumeIds {
if err = doEcDecode(ctx, commandEnv, topologyInfo, *collection, vid); err != nil {
if err = doEcDecode(commandEnv, topologyInfo, *collection, vid); err != nil {
return err
}
}
@ -69,26 +68,26 @@ func (c *commandEcDecode) Do(args []string, commandEnv *CommandEnv, writer io.Wr
return nil
}
func doEcDecode(ctx context.Context, commandEnv *CommandEnv, topoInfo *master_pb.TopologyInfo, collection string, vid needle.VolumeId) (err error) {
func doEcDecode(commandEnv *CommandEnv, topoInfo *master_pb.TopologyInfo, collection string, vid needle.VolumeId) (err error) {
// find volume location
nodeToEcIndexBits := collectEcNodeShardBits(topoInfo, vid)
fmt.Printf("ec volume %d shard locations: %+v\n", vid, nodeToEcIndexBits)
// collect ec shards to the server with most space
targetNodeLocation, err := collectEcShards(ctx, commandEnv, nodeToEcIndexBits, collection, vid)
targetNodeLocation, err := collectEcShards(commandEnv, nodeToEcIndexBits, collection, vid)
if err != nil {
return fmt.Errorf("collectEcShards for volume %d: %v", vid, err)
}
// generate a normal volume
err = generateNormalVolume(ctx, commandEnv.option.GrpcDialOption, needle.VolumeId(vid), collection, targetNodeLocation)
err = generateNormalVolume(commandEnv.option.GrpcDialOption, needle.VolumeId(vid), collection, targetNodeLocation)
if err != nil {
return fmt.Errorf("generate normal volume %d on %s: %v", vid, targetNodeLocation, err)
}
// delete the previous ec shards
err = mountVolumeAndDeleteEcShards(ctx, commandEnv.option.GrpcDialOption, collection, targetNodeLocation, nodeToEcIndexBits, vid)
err = mountVolumeAndDeleteEcShards(commandEnv.option.GrpcDialOption, collection, targetNodeLocation, nodeToEcIndexBits, vid)
if err != nil {
return fmt.Errorf("delete ec shards for volume %d: %v", vid, err)
}
@ -96,11 +95,11 @@ func doEcDecode(ctx context.Context, commandEnv *CommandEnv, topoInfo *master_pb
return nil
}
func mountVolumeAndDeleteEcShards(ctx context.Context, grpcDialOption grpc.DialOption, collection, targetNodeLocation string, nodeToEcIndexBits map[string]erasure_coding.ShardBits, vid needle.VolumeId) error {
func mountVolumeAndDeleteEcShards(grpcDialOption grpc.DialOption, collection, targetNodeLocation string, nodeToEcIndexBits map[string]erasure_coding.ShardBits, vid needle.VolumeId) error {
// mount volume
if err := operation.WithVolumeServerClient(targetNodeLocation, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, mountErr := volumeServerClient.VolumeMount(ctx, &volume_server_pb.VolumeMountRequest{
if err := operation.WithVolumeServerClient(targetNodeLocation, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, mountErr := volumeServerClient.VolumeMount(context.Background(), &volume_server_pb.VolumeMountRequest{
VolumeId: uint32(vid),
})
return mountErr
@ -111,7 +110,7 @@ func mountVolumeAndDeleteEcShards(ctx context.Context, grpcDialOption grpc.DialO
// unmount ec shards
for location, ecIndexBits := range nodeToEcIndexBits {
fmt.Printf("unmount ec volume %d on %s has shards: %+v\n", vid, location, ecIndexBits.ShardIds())
err := unmountEcShards(ctx, grpcDialOption, vid, location, ecIndexBits.ToUint32Slice())
err := unmountEcShards(grpcDialOption, vid, location, ecIndexBits.ToUint32Slice())
if err != nil {
return fmt.Errorf("mountVolumeAndDeleteEcShards unmount ec volume %d on %s: %v", vid, location, err)
}
@ -119,7 +118,7 @@ func mountVolumeAndDeleteEcShards(ctx context.Context, grpcDialOption grpc.DialO
// delete ec shards
for location, ecIndexBits := range nodeToEcIndexBits {
fmt.Printf("delete ec volume %d on %s has shards: %+v\n", vid, location, ecIndexBits.ShardIds())
err := sourceServerDeleteEcShards(ctx, grpcDialOption, collection, vid, location, ecIndexBits.ToUint32Slice())
err := sourceServerDeleteEcShards(grpcDialOption, collection, vid, location, ecIndexBits.ToUint32Slice())
if err != nil {
return fmt.Errorf("mountVolumeAndDeleteEcShards delete ec volume %d on %s: %v", vid, location, err)
}
@ -128,12 +127,12 @@ func mountVolumeAndDeleteEcShards(ctx context.Context, grpcDialOption grpc.DialO
return nil
}
func generateNormalVolume(ctx context.Context, grpcDialOption grpc.DialOption, vid needle.VolumeId, collection string, sourceVolumeServer string) error {
func generateNormalVolume(grpcDialOption grpc.DialOption, vid needle.VolumeId, collection string, sourceVolumeServer string) error {
fmt.Printf("generateNormalVolume from ec volume %d on %s\n", vid, sourceVolumeServer)
err := operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, genErr := volumeServerClient.VolumeEcShardsToVolume(ctx, &volume_server_pb.VolumeEcShardsToVolumeRequest{
err := operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, genErr := volumeServerClient.VolumeEcShardsToVolume(context.Background(), &volume_server_pb.VolumeEcShardsToVolumeRequest{
VolumeId: uint32(vid),
Collection: collection,
})
@ -144,7 +143,7 @@ func generateNormalVolume(ctx context.Context, grpcDialOption grpc.DialOption, v
}
func collectEcShards(ctx context.Context, commandEnv *CommandEnv, nodeToEcIndexBits map[string]erasure_coding.ShardBits, collection string, vid needle.VolumeId) (targetNodeLocation string, err error) {
func collectEcShards(commandEnv *CommandEnv, nodeToEcIndexBits map[string]erasure_coding.ShardBits, collection string, vid needle.VolumeId) (targetNodeLocation string, err error) {
maxShardCount := 0
var exisitngEcIndexBits erasure_coding.ShardBits
@ -170,11 +169,11 @@ func collectEcShards(ctx context.Context, commandEnv *CommandEnv, nodeToEcIndexB
continue
}
err = operation.WithVolumeServerClient(targetNodeLocation, commandEnv.option.GrpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
err = operation.WithVolumeServerClient(targetNodeLocation, commandEnv.option.GrpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
fmt.Printf("copy %d.%v %s => %s\n", vid, needToCopyEcIndexBits.ShardIds(), loc, targetNodeLocation)
_, copyErr := volumeServerClient.VolumeEcShardsCopy(ctx, &volume_server_pb.VolumeEcShardsCopyRequest{
_, copyErr := volumeServerClient.VolumeEcShardsCopy(context.Background(), &volume_server_pb.VolumeEcShardsCopyRequest{
VolumeId: uint32(vid),
Collection: collection,
ShardIds: needToCopyEcIndexBits.ToUint32Slice(),
@ -204,11 +203,11 @@ func collectEcShards(ctx context.Context, commandEnv *CommandEnv, nodeToEcIndexB
}
func collectTopologyInfo(ctx context.Context, commandEnv *CommandEnv) (topoInfo *master_pb.TopologyInfo, err error) {
func collectTopologyInfo(commandEnv *CommandEnv) (topoInfo *master_pb.TopologyInfo, err error) {
var resp *master_pb.VolumeListResponse
err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
resp, err = client.VolumeList(ctx, &master_pb.VolumeListRequest{})
err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
return err
})
if err != nil {

View File

@ -63,22 +63,21 @@ func (c *commandEcEncode) Do(args []string, commandEnv *CommandEnv, writer io.Wr
return nil
}
ctx := context.Background()
vid := needle.VolumeId(*volumeId)
// volumeId is provided
if vid != 0 {
return doEcEncode(ctx, commandEnv, *collection, vid)
return doEcEncode(commandEnv, *collection, vid)
}
// apply to all volumes in the collection
volumeIds, err := collectVolumeIdsForEcEncode(ctx, commandEnv, *collection, *fullPercentage, *quietPeriod)
volumeIds, err := collectVolumeIdsForEcEncode(commandEnv, *collection, *fullPercentage, *quietPeriod)
if err != nil {
return err
}
fmt.Printf("ec encode volumes: %v\n", volumeIds)
for _, vid := range volumeIds {
if err = doEcEncode(ctx, commandEnv, *collection, vid); err != nil {
if err = doEcEncode(commandEnv, *collection, vid); err != nil {
return err
}
}
@ -86,7 +85,7 @@ func (c *commandEcEncode) Do(args []string, commandEnv *CommandEnv, writer io.Wr
return nil
}
func doEcEncode(ctx context.Context, commandEnv *CommandEnv, collection string, vid needle.VolumeId) (err error) {
func doEcEncode(commandEnv *CommandEnv, collection string, vid needle.VolumeId) (err error) {
// find volume location
locations, found := commandEnv.MasterClient.GetLocations(uint32(vid))
if !found {
@ -96,19 +95,19 @@ func doEcEncode(ctx context.Context, commandEnv *CommandEnv, collection string,
// fmt.Printf("found ec %d shards on %v\n", vid, locations)
// mark the volume as readonly
err = markVolumeReadonly(ctx, commandEnv.option.GrpcDialOption, needle.VolumeId(vid), locations)
err = markVolumeReadonly(commandEnv.option.GrpcDialOption, needle.VolumeId(vid), locations)
if err != nil {
return fmt.Errorf("mark volume %d as readonly on %s: %v", vid, locations[0].Url, err)
}
// generate ec shards
err = generateEcShards(ctx, commandEnv.option.GrpcDialOption, needle.VolumeId(vid), collection, locations[0].Url)
err = generateEcShards(commandEnv.option.GrpcDialOption, needle.VolumeId(vid), collection, locations[0].Url)
if err != nil {
return fmt.Errorf("generate ec shards for volume %d on %s: %v", vid, locations[0].Url, err)
}
// balance the ec shards to current cluster
err = spreadEcShards(ctx, commandEnv, vid, collection, locations)
err = spreadEcShards(context.Background(), commandEnv, vid, collection, locations)
if err != nil {
return fmt.Errorf("spread ec shards for volume %d from %s: %v", vid, locations[0].Url, err)
}
@ -116,12 +115,12 @@ func doEcEncode(ctx context.Context, commandEnv *CommandEnv, collection string,
return nil
}
func markVolumeReadonly(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, locations []wdclient.Location) error {
func markVolumeReadonly(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, locations []wdclient.Location) error {
for _, location := range locations {
err := operation.WithVolumeServerClient(location.Url, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, markErr := volumeServerClient.VolumeMarkReadonly(ctx, &volume_server_pb.VolumeMarkReadonlyRequest{
err := operation.WithVolumeServerClient(location.Url, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, markErr := volumeServerClient.VolumeMarkReadonly(context.Background(), &volume_server_pb.VolumeMarkReadonlyRequest{
VolumeId: uint32(volumeId),
})
return markErr
@ -136,10 +135,10 @@ func markVolumeReadonly(ctx context.Context, grpcDialOption grpc.DialOption, vol
return nil
}
func generateEcShards(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, collection string, sourceVolumeServer string) error {
func generateEcShards(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, collection string, sourceVolumeServer string) error {
err := operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, genErr := volumeServerClient.VolumeEcShardsGenerate(ctx, &volume_server_pb.VolumeEcShardsGenerateRequest{
err := operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, genErr := volumeServerClient.VolumeEcShardsGenerate(context.Background(), &volume_server_pb.VolumeEcShardsGenerateRequest{
VolumeId: uint32(volumeId),
Collection: collection,
})
@ -152,7 +151,7 @@ func generateEcShards(ctx context.Context, grpcDialOption grpc.DialOption, volum
func spreadEcShards(ctx context.Context, commandEnv *CommandEnv, volumeId needle.VolumeId, collection string, existingLocations []wdclient.Location) (err error) {
allEcNodes, totalFreeEcSlots, err := collectEcNodes(ctx, commandEnv, "")
allEcNodes, totalFreeEcSlots, err := collectEcNodes(commandEnv, "")
if err != nil {
return err
}
@ -169,26 +168,26 @@ func spreadEcShards(ctx context.Context, commandEnv *CommandEnv, volumeId needle
allocatedEcIds := balancedEcDistribution(allocatedDataNodes)
// ask the data nodes to copy from the source volume server
copiedShardIds, err := parallelCopyEcShardsFromSource(ctx, commandEnv.option.GrpcDialOption, allocatedDataNodes, allocatedEcIds, volumeId, collection, existingLocations[0])
copiedShardIds, err := parallelCopyEcShardsFromSource(commandEnv.option.GrpcDialOption, allocatedDataNodes, allocatedEcIds, volumeId, collection, existingLocations[0])
if err != nil {
return err
}
// unmount the to be deleted shards
err = unmountEcShards(ctx, commandEnv.option.GrpcDialOption, volumeId, existingLocations[0].Url, copiedShardIds)
err = unmountEcShards(commandEnv.option.GrpcDialOption, volumeId, existingLocations[0].Url, copiedShardIds)
if err != nil {
return err
}
// ask the source volume server to clean up copied ec shards
err = sourceServerDeleteEcShards(ctx, commandEnv.option.GrpcDialOption, collection, volumeId, existingLocations[0].Url, copiedShardIds)
err = sourceServerDeleteEcShards(commandEnv.option.GrpcDialOption, collection, volumeId, existingLocations[0].Url, copiedShardIds)
if err != nil {
return fmt.Errorf("source delete copied ecShards %s %d.%v: %v", existingLocations[0].Url, volumeId, copiedShardIds, err)
}
// ask the source volume server to delete the original volume
for _, location := range existingLocations {
err = deleteVolume(ctx, commandEnv.option.GrpcDialOption, volumeId, location.Url)
err = deleteVolume(commandEnv.option.GrpcDialOption, volumeId, location.Url)
if err != nil {
return fmt.Errorf("deleteVolume %s volume %d: %v", location.Url, volumeId, err)
}
@ -198,9 +197,7 @@ func spreadEcShards(ctx context.Context, commandEnv *CommandEnv, volumeId needle
}
func parallelCopyEcShardsFromSource(ctx context.Context, grpcDialOption grpc.DialOption,
targetServers []*EcNode, allocatedEcIds [][]uint32,
volumeId needle.VolumeId, collection string, existingLocation wdclient.Location) (actuallyCopied []uint32, err error) {
func parallelCopyEcShardsFromSource(grpcDialOption grpc.DialOption, targetServers []*EcNode, allocatedEcIds [][]uint32, volumeId needle.VolumeId, collection string, existingLocation wdclient.Location) (actuallyCopied []uint32, err error) {
// parallelize
shardIdChan := make(chan []uint32, len(targetServers))
@ -213,7 +210,7 @@ func parallelCopyEcShardsFromSource(ctx context.Context, grpcDialOption grpc.Dia
wg.Add(1)
go func(server *EcNode, allocatedEcShardIds []uint32) {
defer wg.Done()
copiedShardIds, copyErr := oneServerCopyAndMountEcShardsFromSource(ctx, grpcDialOption, server,
copiedShardIds, copyErr := oneServerCopyAndMountEcShardsFromSource(grpcDialOption, server,
allocatedEcShardIds, volumeId, collection, existingLocation.Url)
if copyErr != nil {
err = copyErr
@ -255,11 +252,11 @@ func balancedEcDistribution(servers []*EcNode) (allocated [][]uint32) {
return allocated
}
func collectVolumeIdsForEcEncode(ctx context.Context, commandEnv *CommandEnv, selectedCollection string, fullPercentage float64, quietPeriod time.Duration) (vids []needle.VolumeId, err error) {
func collectVolumeIdsForEcEncode(commandEnv *CommandEnv, selectedCollection string, fullPercentage float64, quietPeriod time.Duration) (vids []needle.VolumeId, err error) {
var resp *master_pb.VolumeListResponse
err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
resp, err = client.VolumeList(ctx, &master_pb.VolumeListRequest{})
err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
return err
})
if err != nil {

View File

@ -64,7 +64,7 @@ func (c *commandEcRebuild) Do(args []string, commandEnv *CommandEnv, writer io.W
}
// collect all ec nodes
allEcNodes, _, err := collectEcNodes(context.Background(), commandEnv, "")
allEcNodes, _, err := collectEcNodes(commandEnv, "")
if err != nil {
return err
}
@ -92,8 +92,6 @@ func (c *commandEcRebuild) Do(args []string, commandEnv *CommandEnv, writer io.W
func rebuildEcVolumes(commandEnv *CommandEnv, allEcNodes []*EcNode, collection string, writer io.Writer, applyChanges bool) error {
ctx := context.Background()
fmt.Printf("rebuildEcVolumes %s\n", collection)
// collect vid => each shard locations, similar to ecShardMap in topology.go
@ -117,7 +115,7 @@ func rebuildEcVolumes(commandEnv *CommandEnv, allEcNodes []*EcNode, collection s
return fmt.Errorf("disk space is not enough")
}
if err := rebuildOneEcVolume(ctx, commandEnv, allEcNodes[0], collection, vid, locations, writer, applyChanges); err != nil {
if err := rebuildOneEcVolume(commandEnv, allEcNodes[0], collection, vid, locations, writer, applyChanges); err != nil {
return err
}
}
@ -125,13 +123,13 @@ func rebuildEcVolumes(commandEnv *CommandEnv, allEcNodes []*EcNode, collection s
return nil
}
func rebuildOneEcVolume(ctx context.Context, commandEnv *CommandEnv, rebuilder *EcNode, collection string, volumeId needle.VolumeId, locations EcShardLocations, writer io.Writer, applyChanges bool) error {
func rebuildOneEcVolume(commandEnv *CommandEnv, rebuilder *EcNode, collection string, volumeId needle.VolumeId, locations EcShardLocations, writer io.Writer, applyChanges bool) error {
fmt.Printf("rebuildOneEcVolume %s %d\n", collection, volumeId)
// collect shard files to rebuilder local disk
var generatedShardIds []uint32
copiedShardIds, _, err := prepareDataToRecover(ctx, commandEnv, rebuilder, collection, volumeId, locations, writer, applyChanges)
copiedShardIds, _, err := prepareDataToRecover(commandEnv, rebuilder, collection, volumeId, locations, writer, applyChanges)
if err != nil {
return err
}
@ -139,7 +137,7 @@ func rebuildOneEcVolume(ctx context.Context, commandEnv *CommandEnv, rebuilder *
// clean up working files
// ask the rebuilder to delete the copied shards
err = sourceServerDeleteEcShards(ctx, commandEnv.option.GrpcDialOption, collection, volumeId, rebuilder.info.Id, copiedShardIds)
err = sourceServerDeleteEcShards(commandEnv.option.GrpcDialOption, collection, volumeId, rebuilder.info.Id, copiedShardIds)
if err != nil {
fmt.Fprintf(writer, "%s delete copied ec shards %s %d.%v\n", rebuilder.info.Id, collection, volumeId, copiedShardIds)
}
@ -151,13 +149,13 @@ func rebuildOneEcVolume(ctx context.Context, commandEnv *CommandEnv, rebuilder *
}
// generate ec shards, and maybe ecx file
generatedShardIds, err = generateMissingShards(ctx, commandEnv.option.GrpcDialOption, collection, volumeId, rebuilder.info.Id)
generatedShardIds, err = generateMissingShards(commandEnv.option.GrpcDialOption, collection, volumeId, rebuilder.info.Id)
if err != nil {
return err
}
// mount the generated shards
err = mountEcShards(ctx, commandEnv.option.GrpcDialOption, collection, volumeId, rebuilder.info.Id, generatedShardIds)
err = mountEcShards(commandEnv.option.GrpcDialOption, collection, volumeId, rebuilder.info.Id, generatedShardIds)
if err != nil {
return err
}
@ -167,11 +165,10 @@ func rebuildOneEcVolume(ctx context.Context, commandEnv *CommandEnv, rebuilder *
return nil
}
func generateMissingShards(ctx context.Context, grpcDialOption grpc.DialOption,
collection string, volumeId needle.VolumeId, sourceLocation string) (rebuiltShardIds []uint32, err error) {
func generateMissingShards(grpcDialOption grpc.DialOption, collection string, volumeId needle.VolumeId, sourceLocation string) (rebuiltShardIds []uint32, err error) {
err = operation.WithVolumeServerClient(sourceLocation, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
resp, rebultErr := volumeServerClient.VolumeEcShardsRebuild(ctx, &volume_server_pb.VolumeEcShardsRebuildRequest{
err = operation.WithVolumeServerClient(sourceLocation, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
resp, rebultErr := volumeServerClient.VolumeEcShardsRebuild(context.Background(), &volume_server_pb.VolumeEcShardsRebuildRequest{
VolumeId: uint32(volumeId),
Collection: collection,
})
@ -183,7 +180,7 @@ func generateMissingShards(ctx context.Context, grpcDialOption grpc.DialOption,
return
}
func prepareDataToRecover(ctx context.Context, commandEnv *CommandEnv, rebuilder *EcNode, collection string, volumeId needle.VolumeId, locations EcShardLocations, writer io.Writer, applyBalancing bool) (copiedShardIds []uint32, localShardIds []uint32, err error) {
func prepareDataToRecover(commandEnv *CommandEnv, rebuilder *EcNode, collection string, volumeId needle.VolumeId, locations EcShardLocations, writer io.Writer, applyBalancing bool) (copiedShardIds []uint32, localShardIds []uint32, err error) {
needEcxFile := true
var localShardBits erasure_coding.ShardBits
@ -209,8 +206,8 @@ func prepareDataToRecover(ctx context.Context, commandEnv *CommandEnv, rebuilder
var copyErr error
if applyBalancing {
copyErr = operation.WithVolumeServerClient(rebuilder.info.Id, commandEnv.option.GrpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, copyErr := volumeServerClient.VolumeEcShardsCopy(ctx, &volume_server_pb.VolumeEcShardsCopyRequest{
copyErr = operation.WithVolumeServerClient(rebuilder.info.Id, commandEnv.option.GrpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, copyErr := volumeServerClient.VolumeEcShardsCopy(context.Background(), &volume_server_pb.VolumeEcShardsCopyRequest{
VolumeId: uint32(volumeId),
Collection: collection,
ShardIds: []uint32{uint32(shardId)},

View File

@ -121,7 +121,7 @@ func TestCommandEcBalanceVolumeEvenButRackUneven(t *testing.T) {
racks := collectRacks(allEcNodes)
balanceEcVolumes(nil, "c1", allEcNodes, racks, false)
balanceEcRacks(context.Background(), nil, racks, false)
balanceEcRacks(nil, racks, false)
}
func newEcNode(dc string, rack string, dataNodeId string, freeEcSlot int) *EcNode {

View File

@ -38,21 +38,19 @@ func (c *commandFsCat) Do(args []string, commandEnv *CommandEnv, writer io.Write
return err
}
ctx := context.Background()
if commandEnv.isDirectory(ctx, filerServer, filerPort, path) {
if commandEnv.isDirectory(filerServer, filerPort, path) {
return fmt.Errorf("%s is a directory", path)
}
dir, name := filer2.FullPath(path).DirAndName()
return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
return commandEnv.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.LookupDirectoryEntryRequest{
Name: name,
Directory: dir,
}
respLookupEntry, err := client.LookupDirectoryEntry(ctx, request)
respLookupEntry, err := client.LookupDirectoryEntry(context.Background(), request)
if err != nil {
return err
}

View File

@ -1,7 +1,6 @@
package shell
import (
"context"
"io"
)
@ -45,9 +44,7 @@ func (c *commandFsCd) Do(args []string, commandEnv *CommandEnv, writer io.Writer
return nil
}
ctx := context.Background()
err = commandEnv.checkDirectory(ctx, filerServer, filerPort, path)
err = commandEnv.checkDirectory(filerServer, filerPort, path)
if err == nil {
commandEnv.option.FilerHost = filerServer

View File

@ -1,7 +1,6 @@
package shell
import (
"context"
"fmt"
"io"
@ -39,15 +38,13 @@ func (c *commandFsDu) Do(args []string, commandEnv *CommandEnv, writer io.Writer
return err
}
ctx := context.Background()
if commandEnv.isDirectory(ctx, filerServer, filerPort, path) {
if commandEnv.isDirectory(filerServer, filerPort, path) {
path = path + "/"
}
var blockCount, byteCount uint64
dir, name := filer2.FullPath(path).DirAndName()
blockCount, byteCount, err = duTraverseDirectory(ctx, writer, commandEnv.getFilerClient(filerServer, filerPort), dir, name)
blockCount, byteCount, err = duTraverseDirectory(writer, commandEnv.getFilerClient(filerServer, filerPort), dir, name)
if name == "" && err == nil {
fmt.Fprintf(writer, "block:%4d\tbyte:%10d\t%s\n", blockCount, byteCount, dir)
@ -57,15 +54,15 @@ func (c *commandFsDu) Do(args []string, commandEnv *CommandEnv, writer io.Writer
}
func duTraverseDirectory(ctx context.Context, writer io.Writer, filerClient filer2.FilerClient, dir, name string) (blockCount uint64, byteCount uint64, err error) {
func duTraverseDirectory(writer io.Writer, filerClient filer2.FilerClient, dir, name string) (blockCount, byteCount uint64, err error) {
err = filer2.ReadDirAllEntries(ctx, filerClient, filer2.FullPath(dir), name, func(entry *filer_pb.Entry, isLast bool) {
err = filer2.ReadDirAllEntries(filerClient, filer2.FullPath(dir), name, func(entry *filer_pb.Entry, isLast bool) {
if entry.IsDirectory {
subDir := fmt.Sprintf("%s/%s", dir, entry.Name)
if dir == "/" {
subDir = "/" + entry.Name
}
numBlock, numByte, err := duTraverseDirectory(ctx, writer, filerClient, subDir, "")
numBlock, numByte, err := duTraverseDirectory(writer, filerClient, subDir, "")
if err == nil {
blockCount += numBlock
byteCount += numByte
@ -82,12 +79,12 @@ func duTraverseDirectory(ctx context.Context, writer io.Writer, filerClient file
return
}
func (env *CommandEnv) withFilerClient(ctx context.Context, filerServer string, filerPort int64, fn func(context.Context, filer_pb.SeaweedFilerClient) error) error {
func (env *CommandEnv) withFilerClient(filerServer string, filerPort int64, fn func(filer_pb.SeaweedFilerClient) error) error {
filerGrpcAddress := fmt.Sprintf("%s:%d", filerServer, filerPort+10000)
return util.WithCachedGrpcClient(ctx, func(ctx2 context.Context, grpcConnection *grpc.ClientConn) error {
return util.WithCachedGrpcClient(func(grpcConnection *grpc.ClientConn) error {
client := filer_pb.NewSeaweedFilerClient(grpcConnection)
return fn(ctx2, client)
return fn(client)
}, filerGrpcAddress, env.option.GrpcDialOption)
}
@ -105,6 +102,6 @@ func (env *CommandEnv) getFilerClient(filerServer string, filerPort int64) *comm
filerPort: filerPort,
}
}
func (c *commandFilerClient) WithFilerClient(ctx context.Context, fn func(context.Context, filer_pb.SeaweedFilerClient) error) error {
return c.env.withFilerClient(ctx, c.filerServer, c.filerPort, fn)
func (c *commandFilerClient) WithFilerClient(fn func(filer_pb.SeaweedFilerClient) error) error {
return c.env.withFilerClient(c.filerServer, c.filerPort, fn)
}

View File

@ -1,7 +1,6 @@
package shell
import (
"context"
"fmt"
"io"
"os"
@ -60,16 +59,14 @@ func (c *commandFsLs) Do(args []string, commandEnv *CommandEnv, writer io.Writer
return err
}
ctx := context.Background()
if commandEnv.isDirectory(ctx, filerServer, filerPort, path) {
if commandEnv.isDirectory(filerServer, filerPort, path) {
path = path + "/"
}
dir, name := filer2.FullPath(path).DirAndName()
entryCount := 0
err = filer2.ReadDirAllEntries(ctx, commandEnv.getFilerClient(filerServer, filerPort), filer2.FullPath(dir), name, func(entry *filer_pb.Entry, isLast bool) {
err = filer2.ReadDirAllEntries(commandEnv.getFilerClient(filerServer, filerPort), filer2.FullPath(dir), name, func(entry *filer_pb.Entry, isLast bool) {
if !showHidden && strings.HasPrefix(entry.Name, ".") {
return

View File

@ -41,17 +41,15 @@ func (c *commandFsMetaCat) Do(args []string, commandEnv *CommandEnv, writer io.W
return err
}
ctx := context.Background()
dir, name := filer2.FullPath(path).DirAndName()
return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
return commandEnv.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
request := &filer_pb.LookupDirectoryEntryRequest{
Name: name,
Directory: dir,
}
respLookupEntry, err := client.LookupDirectoryEntry(ctx, request)
respLookupEntry, err := client.LookupDirectoryEntry(context.Background(), request)
if err != nil {
return err
}

View File

@ -1,15 +1,15 @@
package shell
import (
"context"
"fmt"
"io"
"os"
"github.com/golang/protobuf/proto"
"github.com/chrislusf/seaweedfs/weed/filer2"
"github.com/chrislusf/seaweedfs/weed/pb/filer_pb"
"github.com/chrislusf/seaweedfs/weed/util"
"github.com/golang/protobuf/proto"
)
func init() {
@ -53,9 +53,7 @@ func (c *commandFsMetaLoad) Do(args []string, commandEnv *CommandEnv, writer io.
var dirCount, fileCount uint64
ctx := context.Background()
err = commandEnv.withFilerClient(ctx, filerServer, filerPort, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
err = commandEnv.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
sizeBuf := make([]byte, 4)
@ -80,7 +78,7 @@ func (c *commandFsMetaLoad) Do(args []string, commandEnv *CommandEnv, writer io.
return err
}
if err := filer_pb.CreateEntry(ctx, client, &filer_pb.CreateEntryRequest{
if err := filer_pb.CreateEntry(client, &filer_pb.CreateEntryRequest{
Directory: fullEntry.Dir,
Entry: fullEntry.Entry,
}); err != nil {

View File

@ -168,7 +168,7 @@ func processOneDirectory(ctx context.Context, writer io.Writer, filerClient file
parentPath filer2.FullPath, queue *util.Queue, jobQueueWg *sync.WaitGroup,
fn func(parentPath filer2.FullPath, entry *filer_pb.Entry)) (err error) {
return filer2.ReadDirAllEntries(ctx, filerClient, parentPath, "", func(entry *filer_pb.Entry, isLast bool) {
return filer2.ReadDirAllEntries(filerClient, parentPath, "", func(entry *filer_pb.Entry, isLast bool) {
fn(parentPath, entry)

View File

@ -47,20 +47,18 @@ func (c *commandFsMv) Do(args []string, commandEnv *CommandEnv, writer io.Writer
return err
}
ctx := context.Background()
sourceDir, sourceName := filer2.FullPath(sourcePath).DirAndName()
destinationDir, destinationName := filer2.FullPath(destinationPath).DirAndName()
return commandEnv.withFilerClient(ctx, filerServer, filerPort, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
return commandEnv.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
// collect destination entry info
destinationRequest := &filer_pb.LookupDirectoryEntryRequest{
Name: destinationDir,
Directory: destinationName,
}
respDestinationLookupEntry, err := client.LookupDirectoryEntry(ctx, destinationRequest)
respDestinationLookupEntry, err := client.LookupDirectoryEntry(context.Background(), destinationRequest)
var targetDir, targetName string
@ -82,7 +80,7 @@ func (c *commandFsMv) Do(args []string, commandEnv *CommandEnv, writer io.Writer
NewName: targetName,
}
_, err = client.AtomicRenameEntry(ctx, request)
_, err = client.AtomicRenameEntry(context.Background(), request)
fmt.Fprintf(writer, "move: %s => %s\n", sourcePath, filer2.NewFullPath(targetDir, targetName))

View File

@ -53,7 +53,7 @@ func treeTraverseDirectory(ctx context.Context, writer io.Writer, filerClient fi
prefix.addMarker(level)
err = filer2.ReadDirAllEntries(ctx, filerClient, dir, name, func(entry *filer_pb.Entry, isLast bool) {
err = filer2.ReadDirAllEntries(filerClient, dir, name, func(entry *filer_pb.Entry, isLast bool) {
if level < 0 && name != "" {
if entry.Name != name {
return

View File

@ -69,9 +69,8 @@ func (c *commandVolumeBalance) Do(args []string, commandEnv *CommandEnv, writer
}
var resp *master_pb.VolumeListResponse
ctx := context.Background()
err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
resp, err = client.VolumeList(ctx, &master_pb.VolumeListRequest{})
err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
return err
})
if err != nil {
@ -239,8 +238,7 @@ func moveVolume(commandEnv *CommandEnv, v *master_pb.VolumeInformationMessage, f
}
fmt.Fprintf(os.Stdout, "moving volume %s%d %s => %s\n", collectionPrefix, v.Id, fullNode.info.Id, emptyNode.info.Id)
if applyBalancing {
ctx := context.Background()
return LiveMoveVolume(ctx, commandEnv.option.GrpcDialOption, needle.VolumeId(v.Id), fullNode.info.Id, emptyNode.info.Id, 5*time.Second)
return LiveMoveVolume(commandEnv.option.GrpcDialOption, needle.VolumeId(v.Id), fullNode.info.Id, emptyNode.info.Id, 5*time.Second)
}
return nil
}

View File

@ -53,9 +53,8 @@ func (c *commandVolumeConfigureReplication) Do(args []string, commandEnv *Comman
replicaPlacementInt32 := uint32(replicaPlacement.Byte())
var resp *master_pb.VolumeListResponse
ctx := context.Background()
err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
resp, err = client.VolumeList(ctx, &master_pb.VolumeListRequest{})
err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
return err
})
if err != nil {
@ -81,8 +80,8 @@ func (c *commandVolumeConfigureReplication) Do(args []string, commandEnv *Comman
}
for _, dst := range allLocations {
err := operation.WithVolumeServerClient(dst.dataNode.Id, commandEnv.option.GrpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
resp, configureErr := volumeServerClient.VolumeConfigure(ctx, &volume_server_pb.VolumeConfigureRequest{
err := operation.WithVolumeServerClient(dst.dataNode.Id, commandEnv.option.GrpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
resp, configureErr := volumeServerClient.VolumeConfigure(context.Background(), &volume_server_pb.VolumeConfigureRequest{
VolumeId: uint32(vid),
Replication: replicaPlacement.String(),
})

View File

@ -1,7 +1,6 @@
package shell
import (
"context"
"fmt"
"io"
@ -47,7 +46,6 @@ func (c *commandVolumeCopy) Do(args []string, commandEnv *CommandEnv, writer io.
return fmt.Errorf("source and target volume servers are the same!")
}
ctx := context.Background()
_, err = copyVolume(ctx, commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer)
_, err = copyVolume(commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer)
return
}

View File

@ -1,7 +1,6 @@
package shell
import (
"context"
"fmt"
"io"
@ -42,7 +41,6 @@ func (c *commandVolumeDelete) Do(args []string, commandEnv *CommandEnv, writer i
return fmt.Errorf("wrong volume id format %s: %v", volumeId, err)
}
ctx := context.Background()
return deleteVolume(ctx, commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer)
return deleteVolume(commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer)
}

View File

@ -50,9 +50,8 @@ func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv,
}
var resp *master_pb.VolumeListResponse
ctx := context.Background()
err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
resp, err = client.VolumeList(ctx, &master_pb.VolumeListRequest{})
err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
return err
})
if err != nil {
@ -113,8 +112,8 @@ func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv,
break
}
err := operation.WithVolumeServerClient(dst.dataNode.Id, commandEnv.option.GrpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, replicateErr := volumeServerClient.VolumeCopy(ctx, &volume_server_pb.VolumeCopyRequest{
err := operation.WithVolumeServerClient(dst.dataNode.Id, commandEnv.option.GrpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, replicateErr := volumeServerClient.VolumeCopy(context.Background(), &volume_server_pb.VolumeCopyRequest{
VolumeId: volumeInfo.Id,
SourceDataNode: sourceNode.dataNode.Id,
})

View File

@ -32,9 +32,8 @@ func (c *commandVolumeList) Help() string {
func (c *commandVolumeList) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
var resp *master_pb.VolumeListResponse
ctx := context.Background()
err = commandEnv.MasterClient.WithClient(ctx, func(client master_pb.SeaweedClient) error {
resp, err = client.VolumeList(ctx, &master_pb.VolumeListRequest{})
err = commandEnv.MasterClient.WithClient(func(client master_pb.SeaweedClient) error {
resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
return err
})
if err != nil {

View File

@ -45,14 +45,13 @@ func (c *commandVolumeMount) Do(args []string, commandEnv *CommandEnv, writer io
return fmt.Errorf("wrong volume id format %s: %v", volumeId, err)
}
ctx := context.Background()
return mountVolume(ctx, commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer)
return mountVolume(commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer)
}
func mountVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer string) (err error) {
return operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, mountErr := volumeServerClient.VolumeMount(ctx, &volume_server_pb.VolumeMountRequest{
func mountVolume(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer string) (err error) {
return operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, mountErr := volumeServerClient.VolumeMount(context.Background(), &volume_server_pb.VolumeMountRequest{
VolumeId: uint32(volumeId),
})
return mountErr

View File

@ -59,26 +59,25 @@ func (c *commandVolumeMove) Do(args []string, commandEnv *CommandEnv, writer io.
return fmt.Errorf("source and target volume servers are the same!")
}
ctx := context.Background()
return LiveMoveVolume(ctx, commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer, 5*time.Second)
return LiveMoveVolume(commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer, 5*time.Second)
}
// LiveMoveVolume moves one volume from one source volume server to one target volume server, with idleTimeout to drain the incoming requests.
func LiveMoveVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer, targetVolumeServer string, idleTimeout time.Duration) (err error) {
func LiveMoveVolume(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer, targetVolumeServer string, idleTimeout time.Duration) (err error) {
log.Printf("copying volume %d from %s to %s", volumeId, sourceVolumeServer, targetVolumeServer)
lastAppendAtNs, err := copyVolume(ctx, grpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer)
lastAppendAtNs, err := copyVolume(grpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer)
if err != nil {
return fmt.Errorf("copy volume %d from %s to %s: %v", volumeId, sourceVolumeServer, targetVolumeServer, err)
}
log.Printf("tailing volume %d from %s to %s", volumeId, sourceVolumeServer, targetVolumeServer)
if err = tailVolume(ctx, grpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer, lastAppendAtNs, idleTimeout); err != nil {
if err = tailVolume(grpcDialOption, volumeId, sourceVolumeServer, targetVolumeServer, lastAppendAtNs, idleTimeout); err != nil {
return fmt.Errorf("tail volume %d from %s to %s: %v", volumeId, sourceVolumeServer, targetVolumeServer, err)
}
log.Printf("deleting volume %d from %s", volumeId, sourceVolumeServer)
if err = deleteVolume(ctx, grpcDialOption, volumeId, sourceVolumeServer); err != nil {
if err = deleteVolume(grpcDialOption, volumeId, sourceVolumeServer); err != nil {
return fmt.Errorf("delete volume %d from %s: %v", volumeId, sourceVolumeServer, err)
}
@ -86,10 +85,10 @@ func LiveMoveVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeI
return nil
}
func copyVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer, targetVolumeServer string) (lastAppendAtNs uint64, err error) {
func copyVolume(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer, targetVolumeServer string) (lastAppendAtNs uint64, err error) {
err = operation.WithVolumeServerClient(targetVolumeServer, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
resp, replicateErr := volumeServerClient.VolumeCopy(ctx, &volume_server_pb.VolumeCopyRequest{
err = operation.WithVolumeServerClient(targetVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
resp, replicateErr := volumeServerClient.VolumeCopy(context.Background(), &volume_server_pb.VolumeCopyRequest{
VolumeId: uint32(volumeId),
SourceDataNode: sourceVolumeServer,
})
@ -102,10 +101,10 @@ func copyVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId ne
return
}
func tailVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer, targetVolumeServer string, lastAppendAtNs uint64, idleTimeout time.Duration) (err error) {
func tailVolume(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer, targetVolumeServer string, lastAppendAtNs uint64, idleTimeout time.Duration) (err error) {
return operation.WithVolumeServerClient(targetVolumeServer, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, replicateErr := volumeServerClient.VolumeTailReceiver(ctx, &volume_server_pb.VolumeTailReceiverRequest{
return operation.WithVolumeServerClient(targetVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, replicateErr := volumeServerClient.VolumeTailReceiver(context.Background(), &volume_server_pb.VolumeTailReceiverRequest{
VolumeId: uint32(volumeId),
SinceNs: lastAppendAtNs,
IdleTimeoutSeconds: uint32(idleTimeout.Seconds()),
@ -116,9 +115,9 @@ func tailVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId ne
}
func deleteVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer string) (err error) {
return operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, deleteErr := volumeServerClient.VolumeDelete(ctx, &volume_server_pb.VolumeDeleteRequest{
func deleteVolume(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer string) (err error) {
return operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, deleteErr := volumeServerClient.VolumeDelete(context.Background(), &volume_server_pb.VolumeDeleteRequest{
VolumeId: uint32(volumeId),
})
return deleteErr

View File

@ -49,18 +49,17 @@ func (c *commandVolumeTierDownload) Do(args []string, commandEnv *CommandEnv, wr
return nil
}
ctx := context.Background()
vid := needle.VolumeId(*volumeId)
// collect topology information
topologyInfo, err := collectTopologyInfo(ctx, commandEnv)
topologyInfo, err := collectTopologyInfo(commandEnv)
if err != nil {
return err
}
// volumeId is provided
if vid != 0 {
return doVolumeTierDownload(ctx, commandEnv, writer, *collection, vid)
return doVolumeTierDownload(commandEnv, writer, *collection, vid)
}
// apply to all volumes in the collection
@ -71,7 +70,7 @@ func (c *commandVolumeTierDownload) Do(args []string, commandEnv *CommandEnv, wr
}
fmt.Printf("tier download volumes: %v\n", volumeIds)
for _, vid := range volumeIds {
if err = doVolumeTierDownload(ctx, commandEnv, writer, *collection, vid); err != nil {
if err = doVolumeTierDownload(commandEnv, writer, *collection, vid); err != nil {
return err
}
}
@ -97,7 +96,7 @@ func collectRemoteVolumes(topoInfo *master_pb.TopologyInfo, selectedCollection s
return
}
func doVolumeTierDownload(ctx context.Context, commandEnv *CommandEnv, writer io.Writer, collection string, vid needle.VolumeId) (err error) {
func doVolumeTierDownload(commandEnv *CommandEnv, writer io.Writer, collection string, vid needle.VolumeId) (err error) {
// find volume location
locations, found := commandEnv.MasterClient.GetLocations(uint32(vid))
if !found {
@ -107,7 +106,7 @@ func doVolumeTierDownload(ctx context.Context, commandEnv *CommandEnv, writer io
// TODO parallelize this
for _, loc := range locations {
// copy the .dat file from remote tier to local
err = downloadDatFromRemoteTier(ctx, commandEnv.option.GrpcDialOption, writer, needle.VolumeId(vid), collection, loc.Url)
err = downloadDatFromRemoteTier(commandEnv.option.GrpcDialOption, writer, needle.VolumeId(vid), collection, loc.Url)
if err != nil {
return fmt.Errorf("download dat file for volume %d to %s: %v", vid, loc.Url, err)
}
@ -116,10 +115,10 @@ func doVolumeTierDownload(ctx context.Context, commandEnv *CommandEnv, writer io
return nil
}
func downloadDatFromRemoteTier(ctx context.Context, grpcDialOption grpc.DialOption, writer io.Writer, volumeId needle.VolumeId, collection string, targetVolumeServer string) error {
func downloadDatFromRemoteTier(grpcDialOption grpc.DialOption, writer io.Writer, volumeId needle.VolumeId, collection string, targetVolumeServer string) error {
err := operation.WithVolumeServerClient(targetVolumeServer, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
stream, downloadErr := volumeServerClient.VolumeTierMoveDatFromRemote(ctx, &volume_server_pb.VolumeTierMoveDatFromRemoteRequest{
err := operation.WithVolumeServerClient(targetVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
stream, downloadErr := volumeServerClient.VolumeTierMoveDatFromRemote(context.Background(), &volume_server_pb.VolumeTierMoveDatFromRemoteRequest{
VolumeId: uint32(volumeId),
Collection: collection,
})
@ -145,14 +144,14 @@ func downloadDatFromRemoteTier(ctx context.Context, grpcDialOption grpc.DialOpti
return downloadErr
}
_, unmountErr := volumeServerClient.VolumeUnmount(ctx, &volume_server_pb.VolumeUnmountRequest{
_, unmountErr := volumeServerClient.VolumeUnmount(context.Background(), &volume_server_pb.VolumeUnmountRequest{
VolumeId: uint32(volumeId),
})
if unmountErr != nil {
return unmountErr
}
_, mountErr := volumeServerClient.VolumeMount(ctx, &volume_server_pb.VolumeMountRequest{
_, mountErr := volumeServerClient.VolumeMount(context.Background(), &volume_server_pb.VolumeMountRequest{
VolumeId: uint32(volumeId),
})
if mountErr != nil {

View File

@ -67,23 +67,22 @@ func (c *commandVolumeTierUpload) Do(args []string, commandEnv *CommandEnv, writ
return nil
}
ctx := context.Background()
vid := needle.VolumeId(*volumeId)
// volumeId is provided
if vid != 0 {
return doVolumeTierUpload(ctx, commandEnv, writer, *collection, vid, *dest, *keepLocalDatFile)
return doVolumeTierUpload(commandEnv, writer, *collection, vid, *dest, *keepLocalDatFile)
}
// apply to all volumes in the collection
// reusing collectVolumeIdsForEcEncode for now
volumeIds, err := collectVolumeIdsForEcEncode(ctx, commandEnv, *collection, *fullPercentage, *quietPeriod)
volumeIds, err := collectVolumeIdsForEcEncode(commandEnv, *collection, *fullPercentage, *quietPeriod)
if err != nil {
return err
}
fmt.Printf("tier upload volumes: %v\n", volumeIds)
for _, vid := range volumeIds {
if err = doVolumeTierUpload(ctx, commandEnv, writer, *collection, vid, *dest, *keepLocalDatFile); err != nil {
if err = doVolumeTierUpload(commandEnv, writer, *collection, vid, *dest, *keepLocalDatFile); err != nil {
return err
}
}
@ -91,20 +90,20 @@ func (c *commandVolumeTierUpload) Do(args []string, commandEnv *CommandEnv, writ
return nil
}
func doVolumeTierUpload(ctx context.Context, commandEnv *CommandEnv, writer io.Writer, collection string, vid needle.VolumeId, dest string, keepLocalDatFile bool) (err error) {
func doVolumeTierUpload(commandEnv *CommandEnv, writer io.Writer, collection string, vid needle.VolumeId, dest string, keepLocalDatFile bool) (err error) {
// find volume location
locations, found := commandEnv.MasterClient.GetLocations(uint32(vid))
if !found {
return fmt.Errorf("volume %d not found", vid)
}
err = markVolumeReadonly(ctx, commandEnv.option.GrpcDialOption, needle.VolumeId(vid), locations)
err = markVolumeReadonly(commandEnv.option.GrpcDialOption, needle.VolumeId(vid), locations)
if err != nil {
return fmt.Errorf("mark volume %d as readonly on %s: %v", vid, locations[0].Url, err)
}
// copy the .dat file to remote tier
err = uploadDatToRemoteTier(ctx, commandEnv.option.GrpcDialOption, writer, needle.VolumeId(vid), collection, locations[0].Url, dest, keepLocalDatFile)
err = uploadDatToRemoteTier(commandEnv.option.GrpcDialOption, writer, needle.VolumeId(vid), collection, locations[0].Url, dest, keepLocalDatFile)
if err != nil {
return fmt.Errorf("copy dat file for volume %d on %s to %s: %v", vid, locations[0].Url, dest, err)
}
@ -112,10 +111,10 @@ func doVolumeTierUpload(ctx context.Context, commandEnv *CommandEnv, writer io.W
return nil
}
func uploadDatToRemoteTier(ctx context.Context, grpcDialOption grpc.DialOption, writer io.Writer, volumeId needle.VolumeId, collection string, sourceVolumeServer string, dest string, keepLocalDatFile bool) error {
func uploadDatToRemoteTier(grpcDialOption grpc.DialOption, writer io.Writer, volumeId needle.VolumeId, collection string, sourceVolumeServer string, dest string, keepLocalDatFile bool) error {
err := operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
stream, copyErr := volumeServerClient.VolumeTierMoveDatToRemote(ctx, &volume_server_pb.VolumeTierMoveDatToRemoteRequest{
err := operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
stream, copyErr := volumeServerClient.VolumeTierMoveDatToRemote(context.Background(), &volume_server_pb.VolumeTierMoveDatToRemoteRequest{
VolumeId: uint32(volumeId),
Collection: collection,
DestinationBackendName: dest,

View File

@ -45,14 +45,13 @@ func (c *commandVolumeUnmount) Do(args []string, commandEnv *CommandEnv, writer
return fmt.Errorf("wrong volume id format %s: %v", volumeId, err)
}
ctx := context.Background()
return unmountVolume(ctx, commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer)
return unmountVolume(commandEnv.option.GrpcDialOption, volumeId, sourceVolumeServer)
}
func unmountVolume(ctx context.Context, grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer string) (err error) {
return operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, unmountErr := volumeServerClient.VolumeUnmount(ctx, &volume_server_pb.VolumeUnmountRequest{
func unmountVolume(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceVolumeServer string) (err error) {
return operation.WithVolumeServerClient(sourceVolumeServer, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, unmountErr := volumeServerClient.VolumeUnmount(context.Background(), &volume_server_pb.VolumeUnmountRequest{
VolumeId: uint32(volumeId),
})
return unmountErr

View File

@ -44,7 +44,7 @@ var (
func NewCommandEnv(options ShellOptions) *CommandEnv {
return &CommandEnv{
env: make(map[string]string),
MasterClient: wdclient.NewMasterClient(context.Background(),
MasterClient: wdclient.NewMasterClient(
options.GrpcDialOption, "shell", strings.Split(*options.Masters, ",")),
option: options,
}
@ -60,19 +60,19 @@ func (ce *CommandEnv) parseUrl(input string) (filerServer string, filerPort int6
return ce.option.FilerHost, ce.option.FilerPort, input, err
}
func (ce *CommandEnv) isDirectory(ctx context.Context, filerServer string, filerPort int64, path string) bool {
func (ce *CommandEnv) isDirectory(filerServer string, filerPort int64, path string) bool {
return ce.checkDirectory(ctx, filerServer, filerPort, path) == nil
return ce.checkDirectory(filerServer, filerPort, path) == nil
}
func (ce *CommandEnv) checkDirectory(ctx context.Context, filerServer string, filerPort int64, path string) error {
func (ce *CommandEnv) checkDirectory(filerServer string, filerPort int64, path string) error {
dir, name := filer2.FullPath(path).DirAndName()
return ce.withFilerClient(ctx, filerServer, filerPort, func(ctx context.Context, client filer_pb.SeaweedFilerClient) error {
return ce.withFilerClient(filerServer, filerPort, func(client filer_pb.SeaweedFilerClient) error {
resp, lookupErr := client.LookupDirectoryEntry(ctx, &filer_pb.LookupDirectoryEntryRequest{
resp, lookupErr := client.LookupDirectoryEntry(context.Background(), &filer_pb.LookupDirectoryEntryRequest{
Directory: dir,
Name: name,
})

View File

@ -116,7 +116,7 @@ func (s *Store) DestroyEcVolume(vid needle.VolumeId) {
}
}
func (s *Store) ReadEcShardNeedle(ctx context.Context, vid needle.VolumeId, n *needle.Needle) (int, error) {
func (s *Store) ReadEcShardNeedle(vid needle.VolumeId, n *needle.Needle) (int, error) {
for _, location := range s.Locations {
if localEcVolume, found := location.FindEcVolume(vid); found {
@ -133,7 +133,7 @@ func (s *Store) ReadEcShardNeedle(ctx context.Context, vid needle.VolumeId, n *n
if len(intervals) > 1 {
glog.V(3).Infof("ReadEcShardNeedle needle id %s intervals:%+v", n.String(), intervals)
}
bytes, isDeleted, err := s.readEcShardIntervals(ctx, vid, n.Id, localEcVolume, intervals)
bytes, isDeleted, err := s.readEcShardIntervals(vid, n.Id, localEcVolume, intervals)
if err != nil {
return 0, fmt.Errorf("ReadEcShardIntervals: %v", err)
}
@ -152,14 +152,14 @@ func (s *Store) ReadEcShardNeedle(ctx context.Context, vid needle.VolumeId, n *n
return 0, fmt.Errorf("ec shard %d not found", vid)
}
func (s *Store) readEcShardIntervals(ctx context.Context, vid needle.VolumeId, needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, intervals []erasure_coding.Interval) (data []byte, is_deleted bool, err error) {
func (s *Store) readEcShardIntervals(vid needle.VolumeId, needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, intervals []erasure_coding.Interval) (data []byte, is_deleted bool, err error) {
if err = s.cachedLookupEcShardLocations(ctx, ecVolume); err != nil {
if err = s.cachedLookupEcShardLocations(ecVolume); err != nil {
return nil, false, fmt.Errorf("failed to locate shard via master grpc %s: %v", s.MasterAddress, err)
}
for i, interval := range intervals {
if d, isDeleted, e := s.readOneEcShardInterval(ctx, needleId, ecVolume, interval); e != nil {
if d, isDeleted, e := s.readOneEcShardInterval(needleId, ecVolume, interval); e != nil {
return nil, isDeleted, e
} else {
if isDeleted {
@ -175,7 +175,7 @@ func (s *Store) readEcShardIntervals(ctx context.Context, vid needle.VolumeId, n
return
}
func (s *Store) readOneEcShardInterval(ctx context.Context, needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, interval erasure_coding.Interval) (data []byte, is_deleted bool, err error) {
func (s *Store) readOneEcShardInterval(needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, interval erasure_coding.Interval) (data []byte, is_deleted bool, err error) {
shardId, actualOffset := interval.ToShardIdAndOffset(erasure_coding.ErasureCodingLargeBlockSize, erasure_coding.ErasureCodingSmallBlockSize)
data = make([]byte, interval.Size)
if shard, found := ecVolume.FindEcVolumeShard(shardId); found {
@ -190,7 +190,7 @@ func (s *Store) readOneEcShardInterval(ctx context.Context, needleId types.Needl
// try reading directly
if hasShardIdLocation {
_, is_deleted, err = s.readRemoteEcShardInterval(ctx, sourceDataNodes, needleId, ecVolume.VolumeId, shardId, data, actualOffset)
_, is_deleted, err = s.readRemoteEcShardInterval(sourceDataNodes, needleId, ecVolume.VolumeId, shardId, data, actualOffset)
if err == nil {
return
}
@ -199,7 +199,7 @@ func (s *Store) readOneEcShardInterval(ctx context.Context, needleId types.Needl
}
// try reading by recovering from other shards
_, is_deleted, err = s.recoverOneRemoteEcShardInterval(ctx, needleId, ecVolume, shardId, data, actualOffset)
_, is_deleted, err = s.recoverOneRemoteEcShardInterval(needleId, ecVolume, shardId, data, actualOffset)
if err == nil {
return
}
@ -215,7 +215,7 @@ func forgetShardId(ecVolume *erasure_coding.EcVolume, shardId erasure_coding.Sha
ecVolume.ShardLocationsLock.Unlock()
}
func (s *Store) cachedLookupEcShardLocations(ctx context.Context, ecVolume *erasure_coding.EcVolume) (err error) {
func (s *Store) cachedLookupEcShardLocations(ecVolume *erasure_coding.EcVolume) (err error) {
shardCount := len(ecVolume.ShardLocations)
if shardCount < erasure_coding.DataShardsCount &&
@ -230,11 +230,11 @@ func (s *Store) cachedLookupEcShardLocations(ctx context.Context, ecVolume *eras
glog.V(3).Infof("lookup and cache ec volume %d locations", ecVolume.VolumeId)
err = operation.WithMasterServerClient(s.MasterAddress, s.grpcDialOption, func(ctx context.Context, masterClient master_pb.SeaweedClient) error {
err = operation.WithMasterServerClient(s.MasterAddress, s.grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
req := &master_pb.LookupEcVolumeRequest{
VolumeId: uint32(ecVolume.VolumeId),
}
resp, err := masterClient.LookupEcVolume(ctx, req)
resp, err := masterClient.LookupEcVolume(context.Background(), req)
if err != nil {
return fmt.Errorf("lookup ec volume %d: %v", ecVolume.VolumeId, err)
}
@ -258,7 +258,7 @@ func (s *Store) cachedLookupEcShardLocations(ctx context.Context, ecVolume *eras
return
}
func (s *Store) readRemoteEcShardInterval(ctx context.Context, sourceDataNodes []string, needleId types.NeedleId, vid needle.VolumeId, shardId erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
func (s *Store) readRemoteEcShardInterval(sourceDataNodes []string, needleId types.NeedleId, vid needle.VolumeId, shardId erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
if len(sourceDataNodes) == 0 {
return 0, false, fmt.Errorf("failed to find ec shard %d.%d", vid, shardId)
@ -266,7 +266,7 @@ func (s *Store) readRemoteEcShardInterval(ctx context.Context, sourceDataNodes [
for _, sourceDataNode := range sourceDataNodes {
glog.V(3).Infof("read remote ec shard %d.%d from %s", vid, shardId, sourceDataNode)
n, is_deleted, err = s.doReadRemoteEcShardInterval(ctx, sourceDataNode, needleId, vid, shardId, buf, offset)
n, is_deleted, err = s.doReadRemoteEcShardInterval(sourceDataNode, needleId, vid, shardId, buf, offset)
if err == nil {
return
}
@ -276,12 +276,12 @@ func (s *Store) readRemoteEcShardInterval(ctx context.Context, sourceDataNodes [
return
}
func (s *Store) doReadRemoteEcShardInterval(ctx context.Context, sourceDataNode string, needleId types.NeedleId, vid needle.VolumeId, shardId erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
func (s *Store) doReadRemoteEcShardInterval(sourceDataNode string, needleId types.NeedleId, vid needle.VolumeId, shardId erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
err = operation.WithVolumeServerClient(sourceDataNode, s.grpcDialOption, func(ctx context.Context, client volume_server_pb.VolumeServerClient) error {
err = operation.WithVolumeServerClient(sourceDataNode, s.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
// copy data slice
shardReadClient, err := client.VolumeEcShardRead(ctx, &volume_server_pb.VolumeEcShardReadRequest{
shardReadClient, err := client.VolumeEcShardRead(context.Background(), &volume_server_pb.VolumeEcShardReadRequest{
VolumeId: uint32(vid),
ShardId: uint32(shardId),
Offset: offset,
@ -316,7 +316,7 @@ func (s *Store) doReadRemoteEcShardInterval(ctx context.Context, sourceDataNode
return
}
func (s *Store) recoverOneRemoteEcShardInterval(ctx context.Context, needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, shardIdToRecover erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
func (s *Store) recoverOneRemoteEcShardInterval(needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, shardIdToRecover erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
glog.V(3).Infof("recover ec shard %d.%d from other locations", ecVolume.VolumeId, shardIdToRecover)
enc, err := reedsolomon.New(erasure_coding.DataShardsCount, erasure_coding.ParityShardsCount)
@ -344,7 +344,7 @@ func (s *Store) recoverOneRemoteEcShardInterval(ctx context.Context, needleId ty
go func(shardId erasure_coding.ShardId, locations []string) {
defer wg.Done()
data := make([]byte, len(buf))
nRead, isDeleted, readErr := s.readRemoteEcShardInterval(ctx, locations, needleId, ecVolume.VolumeId, shardId, data, offset)
nRead, isDeleted, readErr := s.readRemoteEcShardInterval(locations, needleId, ecVolume.VolumeId, shardId, data, offset)
if readErr != nil {
glog.V(3).Infof("recover: readRemoteEcShardInterval %d.%d %d bytes from %+v: %v", ecVolume.VolumeId, shardId, nRead, locations, readErr)
forgetShardId(ecVolume, shardId)

View File

@ -14,7 +14,7 @@ import (
func (s *Store) DeleteEcShardNeedle(ctx context.Context, ecVolume *erasure_coding.EcVolume, n *needle.Needle, cookie types.Cookie) (int64, error) {
count, err := s.ReadEcShardNeedle(ctx, ecVolume.VolumeId, n)
count, err := s.ReadEcShardNeedle(ecVolume.VolumeId, n)
if err != nil {
return 0, err
@ -24,7 +24,7 @@ func (s *Store) DeleteEcShardNeedle(ctx context.Context, ecVolume *erasure_codin
return 0, fmt.Errorf("unexpected cookie %x", cookie)
}
if err = s.doDeleteNeedleFromAtLeastOneRemoteEcShards(ctx, ecVolume, n.Id); err != nil {
if err = s.doDeleteNeedleFromAtLeastOneRemoteEcShards(ecVolume, n.Id); err != nil {
return 0, err
}
@ -32,7 +32,7 @@ func (s *Store) DeleteEcShardNeedle(ctx context.Context, ecVolume *erasure_codin
}
func (s *Store) doDeleteNeedleFromAtLeastOneRemoteEcShards(ctx context.Context, ecVolume *erasure_coding.EcVolume, needleId types.NeedleId) error {
func (s *Store) doDeleteNeedleFromAtLeastOneRemoteEcShards(ecVolume *erasure_coding.EcVolume, needleId types.NeedleId) error {
_, _, intervals, err := ecVolume.LocateEcShardNeedle(needleId, ecVolume.Version)
@ -43,13 +43,13 @@ func (s *Store) doDeleteNeedleFromAtLeastOneRemoteEcShards(ctx context.Context,
shardId, _ := intervals[0].ToShardIdAndOffset(erasure_coding.ErasureCodingLargeBlockSize, erasure_coding.ErasureCodingSmallBlockSize)
hasDeletionSuccess := false
err = s.doDeleteNeedleFromRemoteEcShardServers(ctx, shardId, ecVolume, needleId)
err = s.doDeleteNeedleFromRemoteEcShardServers(shardId, ecVolume, needleId)
if err == nil {
hasDeletionSuccess = true
}
for shardId = erasure_coding.DataShardsCount; shardId < erasure_coding.TotalShardsCount; shardId++ {
if parityDeletionError := s.doDeleteNeedleFromRemoteEcShardServers(ctx, shardId, ecVolume, needleId); parityDeletionError == nil {
if parityDeletionError := s.doDeleteNeedleFromRemoteEcShardServers(shardId, ecVolume, needleId); parityDeletionError == nil {
hasDeletionSuccess = true
}
}
@ -62,7 +62,7 @@ func (s *Store) doDeleteNeedleFromAtLeastOneRemoteEcShards(ctx context.Context,
}
func (s *Store) doDeleteNeedleFromRemoteEcShardServers(ctx context.Context, shardId erasure_coding.ShardId, ecVolume *erasure_coding.EcVolume, needleId types.NeedleId) error {
func (s *Store) doDeleteNeedleFromRemoteEcShardServers(shardId erasure_coding.ShardId, ecVolume *erasure_coding.EcVolume, needleId types.NeedleId) error {
ecVolume.ShardLocationsLock.RLock()
sourceDataNodes, hasShardLocations := ecVolume.ShardLocations[shardId]
@ -74,7 +74,7 @@ func (s *Store) doDeleteNeedleFromRemoteEcShardServers(ctx context.Context, shar
for _, sourceDataNode := range sourceDataNodes {
glog.V(4).Infof("delete from remote ec shard %d.%d from %s", ecVolume.VolumeId, shardId, sourceDataNode)
err := s.doDeleteNeedleFromRemoteEcShard(ctx, sourceDataNode, ecVolume.VolumeId, ecVolume.Collection, ecVolume.Version, needleId)
err := s.doDeleteNeedleFromRemoteEcShard(sourceDataNode, ecVolume.VolumeId, ecVolume.Collection, ecVolume.Version, needleId)
if err != nil {
return err
}
@ -85,12 +85,12 @@ func (s *Store) doDeleteNeedleFromRemoteEcShardServers(ctx context.Context, shar
}
func (s *Store) doDeleteNeedleFromRemoteEcShard(ctx context.Context, sourceDataNode string, vid needle.VolumeId, collection string, version needle.Version, needleId types.NeedleId) error {
func (s *Store) doDeleteNeedleFromRemoteEcShard(sourceDataNode string, vid needle.VolumeId, collection string, version needle.Version, needleId types.NeedleId) error {
return operation.WithVolumeServerClient(sourceDataNode, s.grpcDialOption, func(ctx context.Context, client volume_server_pb.VolumeServerClient) error {
return operation.WithVolumeServerClient(sourceDataNode, s.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
// copy data slice
_, err := client.VolumeEcBlobDelete(ctx, &volume_server_pb.VolumeEcBlobDeleteRequest{
_, err := client.VolumeEcBlobDelete(context.Background(), &volume_server_pb.VolumeEcBlobDeleteRequest{
VolumeId: uint32(vid),
Collection: collection,
FileKey: uint64(needleId),

View File

@ -72,9 +72,9 @@ func (v *Volume) IncrementalBackup(volumeServer string, grpcDialOption grpc.Dial
writeOffset := int64(startFromOffset)
err = operation.WithVolumeServerClient(volumeServer, grpcDialOption, func(ctx context.Context, client volume_server_pb.VolumeServerClient) error {
err = operation.WithVolumeServerClient(volumeServer, grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
stream, err := client.VolumeIncrementalCopy(ctx, &volume_server_pb.VolumeIncrementalCopyRequest{
stream, err := client.VolumeIncrementalCopy(context.Background(), &volume_server_pb.VolumeIncrementalCopyRequest{
VolumeId: uint32(v.Id),
SinceNs: appendAtNs,
})

View File

@ -15,7 +15,7 @@ type AllocateVolumeResult struct {
func AllocateVolume(dn *DataNode, grpcDialOption grpc.DialOption, vid needle.VolumeId, option *VolumeGrowOption) error {
return operation.WithVolumeServerClient(dn.Url(), grpcDialOption, func(ctx context.Context, client volume_server_pb.VolumeServerClient) error {
return operation.WithVolumeServerClient(dn.Url(), grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
_, deleteErr := client.AllocateVolume(context.Background(), &volume_server_pb.AllocateVolumeRequest{
VolumeId: uint32(vid),

View File

@ -19,8 +19,8 @@ func batchVacuumVolumeCheck(grpcDialOption grpc.DialOption, vl *VolumeLayout, vi
errCount := int32(0)
for index, dn := range locationlist.list {
go func(index int, url string, vid needle.VolumeId) {
err := operation.WithVolumeServerClient(url, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
resp, err := volumeServerClient.VacuumVolumeCheck(ctx, &volume_server_pb.VacuumVolumeCheckRequest{
err := operation.WithVolumeServerClient(url, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
resp, err := volumeServerClient.VacuumVolumeCheck(context.Background(), &volume_server_pb.VacuumVolumeCheckRequest{
VolumeId: uint32(vid),
})
if err != nil {
@ -63,8 +63,8 @@ func batchVacuumVolumeCompact(grpcDialOption grpc.DialOption, vl *VolumeLayout,
for index, dn := range locationlist.list {
go func(index int, url string, vid needle.VolumeId) {
glog.V(0).Infoln(index, "Start vacuuming", vid, "on", url)
err := operation.WithVolumeServerClient(url, grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, err := volumeServerClient.VacuumVolumeCompact(ctx, &volume_server_pb.VacuumVolumeCompactRequest{
err := operation.WithVolumeServerClient(url, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, err := volumeServerClient.VacuumVolumeCompact(context.Background(), &volume_server_pb.VacuumVolumeCompactRequest{
VolumeId: uint32(vid),
})
return err
@ -93,8 +93,8 @@ func batchVacuumVolumeCommit(grpcDialOption grpc.DialOption, vl *VolumeLayout, v
isCommitSuccess := true
for _, dn := range locationlist.list {
glog.V(0).Infoln("Start Committing vacuum", vid, "on", dn.Url())
err := operation.WithVolumeServerClient(dn.Url(), grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, err := volumeServerClient.VacuumVolumeCommit(ctx, &volume_server_pb.VacuumVolumeCommitRequest{
err := operation.WithVolumeServerClient(dn.Url(), grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, err := volumeServerClient.VacuumVolumeCommit(context.Background(), &volume_server_pb.VacuumVolumeCommitRequest{
VolumeId: uint32(vid),
})
return err
@ -114,8 +114,8 @@ func batchVacuumVolumeCommit(grpcDialOption grpc.DialOption, vl *VolumeLayout, v
func batchVacuumVolumeCleanup(grpcDialOption grpc.DialOption, vl *VolumeLayout, vid needle.VolumeId, locationlist *VolumeLocationList) {
for _, dn := range locationlist.list {
glog.V(0).Infoln("Start cleaning up", vid, "on", dn.Url())
err := operation.WithVolumeServerClient(dn.Url(), grpcDialOption, func(ctx context.Context, volumeServerClient volume_server_pb.VolumeServerClient) error {
_, err := volumeServerClient.VacuumVolumeCleanup(ctx, &volume_server_pb.VacuumVolumeCleanupRequest{
err := operation.WithVolumeServerClient(dn.Url(), grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
_, err := volumeServerClient.VacuumVolumeCleanup(context.Background(), &volume_server_pb.VacuumVolumeCleanupRequest{
VolumeId: uint32(vid),
})
return err

View File

@ -57,14 +57,14 @@ func GrpcDial(ctx context.Context, address string, opts ...grpc.DialOption) (*gr
return grpc.DialContext(ctx, address, options...)
}
func WithCachedGrpcClient(ctx context.Context, fn func(context.Context, *grpc.ClientConn) error, address string, opts ...grpc.DialOption) error {
func WithCachedGrpcClient(fn func(*grpc.ClientConn) error, address string, opts ...grpc.DialOption) error {
grpcClientsLock.Lock()
existingConnection, found := grpcClients[address]
if found {
grpcClientsLock.Unlock()
err := fn(ctx, existingConnection)
err := fn(existingConnection)
if err != nil {
grpcClientsLock.Lock()
delete(grpcClients, address)
@ -74,7 +74,7 @@ func WithCachedGrpcClient(ctx context.Context, fn func(context.Context, *grpc.Cl
return err
}
grpcConnection, err := GrpcDial(ctx, address, opts...)
grpcConnection, err := GrpcDial(context.Background(), address, opts...)
if err != nil {
grpcClientsLock.Unlock()
return fmt.Errorf("fail to dial %s: %v", address, err)
@ -83,7 +83,7 @@ func WithCachedGrpcClient(ctx context.Context, fn func(context.Context, *grpc.Cl
grpcClients[address] = grpcConnection
grpcClientsLock.Unlock()
err = fn(ctx, grpcConnection)
err = fn(grpcConnection)
if err != nil {
grpcClientsLock.Lock()
delete(grpcClients, address)

View File

@ -13,7 +13,6 @@ import (
)
type MasterClient struct {
ctx context.Context
name string
currentMaster string
masters []string
@ -22,9 +21,8 @@ type MasterClient struct {
vidMap
}
func NewMasterClient(ctx context.Context, grpcDialOption grpc.DialOption, clientName string, masters []string) *MasterClient {
func NewMasterClient(grpcDialOption grpc.DialOption, clientName string, masters []string) *MasterClient {
return &MasterClient{
ctx: ctx,
name: clientName,
masters: masters,
grpcDialOption: grpcDialOption,
@ -66,9 +64,9 @@ func (mc *MasterClient) tryAllMasters() {
func (mc *MasterClient) tryConnectToMaster(master string) (nextHintedLeader string) {
glog.V(1).Infof("%s Connecting to master %v", mc.name, master)
gprcErr := withMasterClient(context.Background(), master, mc.grpcDialOption, func(ctx context.Context, client master_pb.SeaweedClient) error {
gprcErr := withMasterClient(master, mc.grpcDialOption, func(client master_pb.SeaweedClient) error {
stream, err := client.KeepConnected(ctx)
stream, err := client.KeepConnected(context.Background())
if err != nil {
glog.V(0).Infof("%s failed to keep connected to %s: %v", mc.name, master, err)
return err
@ -118,22 +116,22 @@ func (mc *MasterClient) tryConnectToMaster(master string) (nextHintedLeader stri
return
}
func withMasterClient(ctx context.Context, master string, grpcDialOption grpc.DialOption, fn func(ctx context.Context, client master_pb.SeaweedClient) error) error {
func withMasterClient(master string, grpcDialOption grpc.DialOption, fn func(client master_pb.SeaweedClient) error) error {
masterGrpcAddress, parseErr := util.ParseServerToGrpcAddress(master)
if parseErr != nil {
return fmt.Errorf("failed to parse master grpc %v: %v", master, parseErr)
}
return util.WithCachedGrpcClient(ctx, func(ctx2 context.Context, grpcConnection *grpc.ClientConn) error {
return util.WithCachedGrpcClient(func(grpcConnection *grpc.ClientConn) error {
client := master_pb.NewSeaweedClient(grpcConnection)
return fn(ctx2, client)
return fn(client)
}, masterGrpcAddress, grpcDialOption)
}
func (mc *MasterClient) WithClient(ctx context.Context, fn func(client master_pb.SeaweedClient) error) error {
return withMasterClient(ctx, mc.currentMaster, mc.grpcDialOption, func(ctx context.Context, client master_pb.SeaweedClient) error {
func (mc *MasterClient) WithClient(fn func(client master_pb.SeaweedClient) error) error {
return withMasterClient(mc.currentMaster, mc.grpcDialOption, func(client master_pb.SeaweedClient) error {
return fn(client)
})
}