Browse Source

avoid reusing context object

fix https://github.com/chrislusf/seaweedfs/issues/1182
Chris Lu 5 years ago
parent
commit
892e726eb9

+ 3 - 3
weed/command/benchmark.go

@@ -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
 		}

+ 15 - 15
weed/command/filer_copy.go

@@ -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)

+ 1 - 1
weed/command/s3.go

@@ -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)

+ 1 - 1
weed/filer2/filer.go

@@ -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,
 	}

+ 10 - 10
weed/filer2/filer_client_util.go

@@ -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)
 		}

+ 4 - 4
weed/filer2/filer_delete_entry.go

@@ -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 {

+ 20 - 20
weed/filesys/dir.go

@@ -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

+ 2 - 2
weed/filesys/dir_link.go

@@ -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
 		}

+ 2 - 2
weed/filesys/dir_rename.go

@@ -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

+ 12 - 12
weed/filesys/dirty_page.go

@@ -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

Some files were not shown because too many files changed in this diff