aboutsummaryrefslogtreecommitdiff
path: root/weed/server
diff options
context:
space:
mode:
Diffstat (limited to 'weed/server')
-rw-r--r--weed/server/common.go26
-rw-r--r--weed/server/filer_grpc_server.go38
-rw-r--r--weed/server/filer_grpc_server_admin.go4
-rw-r--r--weed/server/filer_grpc_server_dlm.go8
-rw-r--r--weed/server/filer_grpc_server_rename.go12
-rw-r--r--weed/server/filer_grpc_server_sub_meta.go36
-rw-r--r--weed/server/filer_grpc_server_traverse_meta.go4
-rw-r--r--weed/server/filer_server.go16
-rw-r--r--weed/server/filer_server_handlers.go12
-rw-r--r--weed/server/filer_server_handlers_proxy.go8
-rw-r--r--weed/server/filer_server_handlers_read.go16
-rw-r--r--weed/server/filer_server_handlers_read_dir.go8
-rw-r--r--weed/server/filer_server_handlers_tagging.go6
-rw-r--r--weed/server/filer_server_handlers_write.go16
-rw-r--r--weed/server/filer_server_handlers_write_autochunk.go20
-rw-r--r--weed/server/filer_server_handlers_write_cipher.go4
-rw-r--r--weed/server/filer_server_handlers_write_merge.go4
-rw-r--r--weed/server/filer_server_handlers_write_upload.go14
-rw-r--r--weed/server/master_grpc_server.go54
-rw-r--r--weed/server/master_grpc_server_admin.go6
-rw-r--r--weed/server/master_grpc_server_assign.go12
-rw-r--r--weed/server/master_grpc_server_volume.go16
-rw-r--r--weed/server/master_server.go40
-rw-r--r--weed/server/master_server_handlers.go4
-rw-r--r--weed/server/master_server_handlers_admin.go8
-rw-r--r--weed/server/raft_hashicorp.go26
-rw-r--r--weed/server/raft_server.go24
-rw-r--r--weed/server/raft_server_handlers.go4
-rw-r--r--weed/server/volume_grpc_admin.go38
-rw-r--r--weed/server/volume_grpc_client_to_master.go50
-rw-r--r--weed/server/volume_grpc_copy.go12
-rw-r--r--weed/server/volume_grpc_erasure_coding.go28
-rw-r--r--weed/server/volume_grpc_query.go8
-rw-r--r--weed/server/volume_grpc_tail.go12
-rw-r--r--weed/server/volume_grpc_vacuum.go18
-rw-r--r--weed/server/volume_server.go12
-rw-r--r--weed/server/volume_server_handlers.go20
-rw-r--r--weed/server/volume_server_handlers_read.go40
-rw-r--r--weed/server/volume_server_handlers_write.go10
-rw-r--r--weed/server/webdav_server.go44
40 files changed, 369 insertions, 369 deletions
diff --git a/weed/server/common.go b/weed/server/common.go
index 5dad9d81b..652f903a3 100644
--- a/weed/server/common.go
+++ b/weed/server/common.go
@@ -22,7 +22,7 @@ import (
"google.golang.org/grpc"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/stats"
"github.com/seaweedfs/seaweedfs/weed/storage/needle"
@@ -74,7 +74,7 @@ func writeJson(w http.ResponseWriter, r *http.Request, httpStatus int, obj inter
}
if httpStatus >= 400 {
- glog.V(0).Infof("response method:%s URL:%s with httpStatus:%d and JSON:%s",
+ log.V(3).Infof("response method:%s URL:%s with httpStatus:%d and JSON:%s",
r.Method, r.URL.String(), httpStatus, string(bytes))
}
@@ -110,19 +110,19 @@ func writeJson(w http.ResponseWriter, r *http.Request, httpStatus int, obj inter
// wrapper for writeJson - just logs errors
func writeJsonQuiet(w http.ResponseWriter, r *http.Request, httpStatus int, obj interface{}) {
if err := writeJson(w, r, httpStatus, obj); err != nil {
- glog.V(0).Infof("error writing JSON status %s %d: %v", r.URL, httpStatus, err)
- glog.V(1).Infof("JSON content: %+v", obj)
+ log.V(3).Infof("error writing JSON status %s %d: %v", r.URL, httpStatus, err)
+ log.V(2).Infof("JSON content: %+v", obj)
}
}
func writeJsonError(w http.ResponseWriter, r *http.Request, httpStatus int, err error) {
m := make(map[string]interface{})
m["error"] = err.Error()
- glog.V(1).Infof("error JSON response status %d: %s", httpStatus, m["error"])
+ log.V(2).Infof("error JSON response status %d: %s", httpStatus, m["error"])
writeJsonQuiet(w, r, httpStatus, m)
}
func debug(params ...interface{}) {
- glog.V(4).Infoln(params...)
+ log.V(-1).Infoln(params...)
}
func submitForClientHandler(w http.ResponseWriter, r *http.Request, masterFn operation.GetMasterFn, grpcDialOption grpc.DialOption) {
@@ -302,13 +302,13 @@ func ProcessRangeRequest(r *http.Request, w http.ResponseWriter, totalSize int64
w.Header().Set("Content-Length", strconv.FormatInt(totalSize, 10))
writeFn, err := prepareWriteFn(0, totalSize)
if err != nil {
- glog.Errorf("ProcessRangeRequest: %v", err)
+ log.Errorf("ProcessRangeRequest: %v", err)
w.Header().Del("Content-Length")
http.Error(w, err.Error(), http.StatusInternalServerError)
return fmt.Errorf("ProcessRangeRequest: %v", err)
}
if err = writeFn(bufferedWriter); err != nil {
- glog.Errorf("ProcessRangeRequest: %v", err)
+ log.Errorf("ProcessRangeRequest: %v", err)
w.Header().Del("Content-Length")
http.Error(w, err.Error(), http.StatusInternalServerError)
return fmt.Errorf("ProcessRangeRequest: %v", err)
@@ -320,7 +320,7 @@ func ProcessRangeRequest(r *http.Request, w http.ResponseWriter, totalSize int64
//mostly copy from src/pkg/net/http/fs.go
ranges, err := parseRange(rangeReq, totalSize)
if err != nil {
- glog.Errorf("ProcessRangeRequest headers: %+v err: %v", w.Header(), err)
+ log.Errorf("ProcessRangeRequest headers: %+v err: %v", w.Header(), err)
http.Error(w, err.Error(), http.StatusRequestedRangeNotSatisfiable)
return fmt.Errorf("ProcessRangeRequest header: %v", err)
}
@@ -352,7 +352,7 @@ func ProcessRangeRequest(r *http.Request, w http.ResponseWriter, totalSize int64
writeFn, err := prepareWriteFn(ra.start, ra.length)
if err != nil {
- glog.Errorf("ProcessRangeRequest range[0]: %+v err: %v", w.Header(), err)
+ log.Errorf("ProcessRangeRequest range[0]: %+v err: %v", w.Header(), err)
w.Header().Del("Content-Length")
http.Error(w, err.Error(), http.StatusInternalServerError)
return fmt.Errorf("ProcessRangeRequest: %v", err)
@@ -360,7 +360,7 @@ func ProcessRangeRequest(r *http.Request, w http.ResponseWriter, totalSize int64
w.WriteHeader(http.StatusPartialContent)
err = writeFn(bufferedWriter)
if err != nil {
- glog.Errorf("ProcessRangeRequest range[0]: %+v err: %v", w.Header(), err)
+ log.Errorf("ProcessRangeRequest range[0]: %+v err: %v", w.Header(), err)
w.Header().Del("Content-Length")
http.Error(w, err.Error(), http.StatusInternalServerError)
return fmt.Errorf("ProcessRangeRequest range[0]: %v", err)
@@ -378,7 +378,7 @@ func ProcessRangeRequest(r *http.Request, w http.ResponseWriter, totalSize int64
}
writeFn, err := prepareWriteFn(ra.start, ra.length)
if err != nil {
- glog.Errorf("ProcessRangeRequest range[%d] err: %v", i, err)
+ log.Errorf("ProcessRangeRequest range[%d] err: %v", i, err)
http.Error(w, "Internal Error", http.StatusInternalServerError)
return fmt.Errorf("ProcessRangeRequest range[%d] err: %v", i, err)
}
@@ -415,7 +415,7 @@ func ProcessRangeRequest(r *http.Request, w http.ResponseWriter, totalSize int64
}
w.WriteHeader(http.StatusPartialContent)
if _, err := io.CopyN(bufferedWriter, sendContent, sendSize); err != nil {
- glog.Errorf("ProcessRangeRequest err: %v", err)
+ log.Errorf("ProcessRangeRequest err: %v", err)
http.Error(w, "Internal Error", http.StatusInternalServerError)
return fmt.Errorf("ProcessRangeRequest err: %v", err)
}
diff --git a/weed/server/filer_grpc_server.go b/weed/server/filer_grpc_server.go
index b1440c94f..cf4bd0396 100644
--- a/weed/server/filer_grpc_server.go
+++ b/weed/server/filer_grpc_server.go
@@ -11,7 +11,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/cluster"
"github.com/seaweedfs/seaweedfs/weed/filer"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
@@ -21,14 +21,14 @@ import (
func (fs *FilerServer) LookupDirectoryEntry(ctx context.Context, req *filer_pb.LookupDirectoryEntryRequest) (*filer_pb.LookupDirectoryEntryResponse, error) {
- glog.V(4).Infof("LookupDirectoryEntry %s", filepath.Join(req.Directory, req.Name))
+ log.V(-1).Infof("LookupDirectoryEntry %s", filepath.Join(req.Directory, req.Name))
entry, err := fs.filer.FindEntry(ctx, util.JoinPath(req.Directory, req.Name))
if err == filer_pb.ErrNotFound {
return &filer_pb.LookupDirectoryEntryResponse{}, err
}
if err != nil {
- glog.V(3).Infof("LookupDirectoryEntry %s: %+v, ", filepath.Join(req.Directory, req.Name), err)
+ log.V(0).Infof("LookupDirectoryEntry %s: %+v, ", filepath.Join(req.Directory, req.Name), err)
return nil, err
}
@@ -39,7 +39,7 @@ func (fs *FilerServer) LookupDirectoryEntry(ctx context.Context, req *filer_pb.L
func (fs *FilerServer) ListEntries(req *filer_pb.ListEntriesRequest, stream filer_pb.SeaweedFiler_ListEntriesServer) (err error) {
- glog.V(4).Infof("ListEntries %v", req)
+ log.V(-1).Infof("ListEntries %v", req)
limit := int(req.Limit)
if limit == 0 {
@@ -97,7 +97,7 @@ func (fs *FilerServer) LookupVolume(ctx context.Context, req *filer_pb.LookupVol
for _, vidString := range req.VolumeIds {
vid, err := strconv.Atoi(vidString)
if err != nil {
- glog.V(1).Infof("Unknown volume id %d", vid)
+ log.V(2).Infof("Unknown volume id %d", vid)
return nil, err
}
var locs []*filer_pb.Location
@@ -138,7 +138,7 @@ func (fs *FilerServer) lookupFileId(fileId string) (targetUrls []string, err err
func (fs *FilerServer) CreateEntry(ctx context.Context, req *filer_pb.CreateEntryRequest) (resp *filer_pb.CreateEntryResponse, err error) {
- glog.V(4).Infof("CreateEntry %v/%v", req.Directory, req.Entry.Name)
+ log.V(-1).Infof("CreateEntry %v/%v", req.Directory, req.Entry.Name)
resp = &filer_pb.CreateEntryResponse{}
@@ -160,7 +160,7 @@ func (fs *FilerServer) CreateEntry(ctx context.Context, req *filer_pb.CreateEntr
if createErr == nil {
fs.filer.DeleteChunksNotRecursive(garbage)
} else {
- glog.V(3).Infof("CreateEntry %s: %v", filepath.Join(req.Directory, req.Entry.Name), createErr)
+ log.V(0).Infof("CreateEntry %s: %v", filepath.Join(req.Directory, req.Entry.Name), createErr)
resp.Error = createErr.Error()
}
@@ -169,7 +169,7 @@ func (fs *FilerServer) CreateEntry(ctx context.Context, req *filer_pb.CreateEntr
func (fs *FilerServer) UpdateEntry(ctx context.Context, req *filer_pb.UpdateEntryRequest) (*filer_pb.UpdateEntryResponse, error) {
- glog.V(4).Infof("UpdateEntry %v", req)
+ log.V(-1).Infof("UpdateEntry %v", req)
fullpath := util.Join(req.Directory, req.Entry.Name)
entry, err := fs.filer.FindEntry(ctx, util.FullPath(fullpath))
@@ -195,7 +195,7 @@ func (fs *FilerServer) UpdateEntry(ctx context.Context, req *filer_pb.UpdateEntr
fs.filer.NotifyUpdateEvent(ctx, entry, newEntry, true, req.IsFromOtherCluster, req.Signatures)
} else {
- glog.V(3).Infof("UpdateEntry %s: %v", filepath.Join(req.Directory, req.Entry.Name), err)
+ log.V(0).Infof("UpdateEntry %s: %v", filepath.Join(req.Directory, req.Entry.Name), err)
}
return &filer_pb.UpdateEntryResponse{}, err
@@ -230,7 +230,7 @@ func (fs *FilerServer) cleanupChunks(fullpath string, existingEntry *filer.Entry
chunks, err = filer.MaybeManifestize(fs.saveAsChunk(so), chunks)
if err != nil {
// not good, but should be ok
- glog.V(0).Infof("MaybeManifestize: %v", err)
+ log.V(3).Infof("MaybeManifestize: %v", err)
}
}
@@ -241,7 +241,7 @@ func (fs *FilerServer) cleanupChunks(fullpath string, existingEntry *filer.Entry
func (fs *FilerServer) AppendToEntry(ctx context.Context, req *filer_pb.AppendToEntryRequest) (*filer_pb.AppendToEntryResponse, error) {
- glog.V(4).Infof("AppendToEntry %v", req)
+ log.V(-1).Infof("AppendToEntry %v", req)
fullpath := util.NewFullPath(req.Directory, req.EntryName)
lockClient := cluster.NewLockClient(fs.grpcDialOption, fs.option.Host)
@@ -273,13 +273,13 @@ func (fs *FilerServer) AppendToEntry(ctx context.Context, req *filer_pb.AppendTo
entry.Chunks = append(entry.GetChunks(), req.Chunks...)
so, err := fs.detectStorageOption(string(fullpath), "", "", entry.TtlSec, "", "", "", "")
if err != nil {
- glog.Warningf("detectStorageOption: %v", err)
+ log.Warningf("detectStorageOption: %v", err)
return &filer_pb.AppendToEntryResponse{}, err
}
entry.Chunks, err = filer.MaybeManifestize(fs.saveAsChunk(so), entry.GetChunks())
if err != nil {
// not good, but should be ok
- glog.V(0).Infof("MaybeManifestize: %v", err)
+ log.V(3).Infof("MaybeManifestize: %v", err)
}
err = fs.filer.CreateEntry(context.Background(), entry, false, false, nil, false, fs.filer.MaxFilenameLength)
@@ -289,7 +289,7 @@ func (fs *FilerServer) AppendToEntry(ctx context.Context, req *filer_pb.AppendTo
func (fs *FilerServer) DeleteEntry(ctx context.Context, req *filer_pb.DeleteEntryRequest) (resp *filer_pb.DeleteEntryResponse, err error) {
- glog.V(4).Infof("DeleteEntry %v", req)
+ log.V(-1).Infof("DeleteEntry %v", req)
err = fs.filer.DeleteEntryMetaAndData(ctx, util.JoinPath(req.Directory, req.Name), req.IsRecursive, req.IgnoreRecursiveError, req.IsDeleteData, req.IsFromOtherCluster, req.Signatures, req.IfNotModifiedAfter)
resp = &filer_pb.DeleteEntryResponse{}
@@ -307,7 +307,7 @@ func (fs *FilerServer) AssignVolume(ctx context.Context, req *filer_pb.AssignVol
so, err := fs.detectStorageOption(req.Path, req.Collection, req.Replication, req.TtlSec, req.DiskType, req.DataCenter, req.Rack, req.DataNode)
if err != nil {
- glog.V(3).Infof("AssignVolume: %v", err)
+ log.V(0).Infof("AssignVolume: %v", err)
return &filer_pb.AssignVolumeResponse{Error: fmt.Sprintf("assign volume: %v", err)}, nil
}
@@ -315,11 +315,11 @@ func (fs *FilerServer) AssignVolume(ctx context.Context, req *filer_pb.AssignVol
assignResult, err := operation.Assign(fs.filer.GetMaster, fs.grpcDialOption, assignRequest, altRequest)
if err != nil {
- glog.V(3).Infof("AssignVolume: %v", err)
+ log.V(0).Infof("AssignVolume: %v", err)
return &filer_pb.AssignVolumeResponse{Error: fmt.Sprintf("assign volume: %v", err)}, nil
}
if assignResult.Error != "" {
- glog.V(3).Infof("AssignVolume error: %v", assignResult.Error)
+ log.V(0).Infof("AssignVolume error: %v", assignResult.Error)
return &filer_pb.AssignVolumeResponse{Error: fmt.Sprintf("assign volume result: %v", assignResult.Error)}, nil
}
@@ -339,7 +339,7 @@ func (fs *FilerServer) AssignVolume(ctx context.Context, req *filer_pb.AssignVol
func (fs *FilerServer) CollectionList(ctx context.Context, req *filer_pb.CollectionListRequest) (resp *filer_pb.CollectionListResponse, err error) {
- glog.V(4).Infof("CollectionList %v", req)
+ log.V(-1).Infof("CollectionList %v", req)
resp = &filer_pb.CollectionListResponse{}
err = fs.filer.MasterClient.WithClient(false, func(client master_pb.SeaweedClient) error {
@@ -361,7 +361,7 @@ func (fs *FilerServer) CollectionList(ctx context.Context, req *filer_pb.Collect
func (fs *FilerServer) DeleteCollection(ctx context.Context, req *filer_pb.DeleteCollectionRequest) (resp *filer_pb.DeleteCollectionResponse, err error) {
- glog.V(4).Infof("DeleteCollection %v", req)
+ log.V(-1).Infof("DeleteCollection %v", req)
err = fs.filer.DoDeleteCollection(req.GetCollection())
diff --git a/weed/server/filer_grpc_server_admin.go b/weed/server/filer_grpc_server_admin.go
index 8b4912258..e9c2c3dde 100644
--- a/weed/server/filer_grpc_server_admin.go
+++ b/weed/server/filer_grpc_server_admin.go
@@ -6,7 +6,7 @@ import (
"time"
"github.com/seaweedfs/seaweedfs/weed/cluster"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
@@ -100,7 +100,7 @@ func (fs *FilerServer) GetFilerConfiguration(ctx context.Context, req *filer_pb.
MinorVersion: util.MINOR_VERSION,
}
- glog.V(4).Infof("GetFilerConfiguration: %v", t)
+ log.V(-1).Infof("GetFilerConfiguration: %v", t)
return t, nil
}
diff --git a/weed/server/filer_grpc_server_dlm.go b/weed/server/filer_grpc_server_dlm.go
index 5ec147835..3396a63ad 100644
--- a/weed/server/filer_grpc_server_dlm.go
+++ b/weed/server/filer_grpc_server_dlm.go
@@ -4,7 +4,7 @@ import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/cluster/lock_manager"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"google.golang.org/grpc/codes"
@@ -20,7 +20,7 @@ func (fs *FilerServer) DistributedLock(ctx context.Context, req *filer_pb.LockRe
var movedTo pb.ServerAddress
expiredAtNs := time.Now().Add(time.Duration(req.SecondsToLock) * time.Second).UnixNano()
resp.LockOwner, resp.RenewToken, movedTo, err = fs.filer.Dlm.LockWithTimeout(req.Name, expiredAtNs, req.RenewToken, req.Owner)
- glog.V(3).Infof("lock %s %v %v %v, isMoved=%v %v", req.Name, req.SecondsToLock, req.RenewToken, req.Owner, req.IsMoved, movedTo)
+ log.V(0).Infof("lock %s %v %v %v, isMoved=%v %v", req.Name, req.SecondsToLock, req.RenewToken, req.Owner, req.IsMoved, movedTo)
if movedTo != "" && movedTo != fs.option.Host && !req.IsMoved {
err = pb.WithFilerClient(false, 0, movedTo, fs.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
secondResp, err := client.DistributedLock(context.Background(), &filer_pb.LockRequest{
@@ -100,7 +100,7 @@ func (fs *FilerServer) FindLockOwner(ctx context.Context, req *filer_pb.FindLock
}
if owner == "" {
- glog.V(0).Infof("find lock %s moved to %v: %v", req.Name, movedTo, err)
+ log.V(3).Infof("find lock %s moved to %v: %v", req.Name, movedTo, err)
return nil, status.Error(codes.NotFound, fmt.Sprintf("lock %s not found", req.Name))
}
if err != nil {
@@ -145,7 +145,7 @@ func (fs *FilerServer) OnDlmChangeSnapshot(snapshot []pb.ServerAddress) {
return err
}); err != nil {
// it may not be worth retrying, since the lock may have expired
- glog.Errorf("transfer lock %v to %v: %v", lock.Key, server, err)
+ log.Errorf("transfer lock %v to %v: %v", lock.Key, server, err)
}
}
diff --git a/weed/server/filer_grpc_server_rename.go b/weed/server/filer_grpc_server_rename.go
index db00dd496..e45c66620 100644
--- a/weed/server/filer_grpc_server_rename.go
+++ b/weed/server/filer_grpc_server_rename.go
@@ -7,14 +7,14 @@ import (
"time"
"github.com/seaweedfs/seaweedfs/weed/filer"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/util"
)
func (fs *FilerServer) AtomicRenameEntry(ctx context.Context, req *filer_pb.AtomicRenameEntryRequest) (*filer_pb.AtomicRenameEntryResponse, error) {
- glog.V(1).Infof("AtomicRenameEntry %v", req)
+ log.V(2).Infof("AtomicRenameEntry %v", req)
oldParent := util.FullPath(filepath.ToSlash(req.OldDirectory))
newParent := util.FullPath(filepath.ToSlash(req.NewDirectory))
@@ -50,7 +50,7 @@ func (fs *FilerServer) AtomicRenameEntry(ctx context.Context, req *filer_pb.Atom
func (fs *FilerServer) StreamRenameEntry(req *filer_pb.StreamRenameEntryRequest, stream filer_pb.SeaweedFiler_StreamRenameEntryServer) (err error) {
- glog.V(1).Infof("StreamRenameEntry %v", req)
+ log.V(2).Infof("StreamRenameEntry %v", req)
oldParent := util.FullPath(filepath.ToSlash(req.OldDirectory))
newParent := util.FullPath(filepath.ToSlash(req.NewDirectory))
@@ -122,7 +122,7 @@ func (fs *FilerServer) moveFolderSubEntries(ctx context.Context, stream filer_pb
currentDirPath := oldParent.Child(entry.Name())
newDirPath := newParent.Child(newName)
- glog.V(1).Infof("moving folder %s => %s", currentDirPath, newDirPath)
+ log.V(2).Infof("moving folder %s => %s", currentDirPath, newDirPath)
lastFileName := ""
includeLastFile := false
@@ -154,10 +154,10 @@ func (fs *FilerServer) moveSelfEntry(ctx context.Context, stream filer_pb.Seawee
oldPath, newPath := oldParent.Child(entry.Name()), newParent.Child(newName)
- glog.V(1).Infof("moving entry %s => %s", oldPath, newPath)
+ log.V(2).Infof("moving entry %s => %s", oldPath, newPath)
if oldPath == newPath {
- glog.V(1).Infof("skip moving entry %s => %s", oldPath, newPath)
+ log.V(2).Infof("skip moving entry %s => %s", oldPath, newPath)
return nil
}
diff --git a/weed/server/filer_grpc_server_sub_meta.go b/weed/server/filer_grpc_server_sub_meta.go
index dfe594b46..583fd06c6 100644
--- a/weed/server/filer_grpc_server_sub_meta.go
+++ b/weed/server/filer_grpc_server_sub_meta.go
@@ -12,7 +12,7 @@ import (
"google.golang.org/protobuf/proto"
"github.com/seaweedfs/seaweedfs/weed/filer"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/util"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
@@ -36,13 +36,13 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
return fmt.Errorf("duplicated subscription detected for client %s id %d", clientName, req.ClientId)
}
defer func() {
- glog.V(0).Infof("disconnect %v subscriber %s clientId:%d", clientName, req.PathPrefix, req.ClientId)
+ log.V(3).Infof("disconnect %v subscriber %s clientId:%d", clientName, req.PathPrefix, req.ClientId)
fs.deleteClient("", clientName, req.ClientId, req.ClientEpoch)
fs.filer.MetaAggregator.ListenersCond.Broadcast() // nudges the subscribers that are waiting
}()
lastReadTime := log_buffer.NewMessagePosition(req.SinceNs, -2)
- glog.V(0).Infof(" %v starts to subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
+ log.V(3).Infof(" %v starts to subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
eachEventNotificationFn := fs.eachEventNotificationFn(req, stream, clientName)
@@ -55,7 +55,7 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
for {
- glog.V(4).Infof("read on disk %v aggregated subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
+ log.V(-1).Infof("read on disk %v aggregated subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
processedTsNs, isDone, readPersistedLogErr = fs.filer.ReadPersistedLogBuffer(lastReadTime, req.UntilNs, eachLogEntryFn)
if readPersistedLogErr != nil {
@@ -65,7 +65,7 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
return nil
}
- glog.V(4).Infof("processed to %v: %v", clientName, processedTsNs)
+ log.V(-1).Infof("processed to %v: %v", clientName, processedTsNs)
if processedTsNs != 0 {
lastReadTime = log_buffer.NewMessagePosition(processedTsNs, -2)
} else {
@@ -80,7 +80,7 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
}
}
- glog.V(4).Infof("read in memory %v aggregated subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
+ log.V(-1).Infof("read in memory %v aggregated subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
lastReadTime, isDone, readInMemoryLogErr = fs.filer.MetaAggregator.MetaLogBuffer.LoopProcessLogData("aggMeta:"+clientName, lastReadTime, req.UntilNs, func() bool {
// Check if the client has disconnected by monitoring the context
@@ -99,7 +99,7 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
if errors.Is(readInMemoryLogErr, log_buffer.ResumeFromDiskError) {
continue
}
- glog.Errorf("processed to %v: %v", lastReadTime, readInMemoryLogErr)
+ log.Errorf("processed to %v: %v", lastReadTime, readInMemoryLogErr)
if !errors.Is(readInMemoryLogErr, log_buffer.ResumeError) {
break
}
@@ -108,7 +108,7 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
return nil
}
if !fs.hasClient(req.ClientId, req.ClientEpoch) {
- glog.V(0).Infof("client %v is closed", clientName)
+ log.V(3).Infof("client %v is closed", clientName)
return nil
}
@@ -134,13 +134,13 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
return fmt.Errorf("duplicated local subscription detected for client %s clientId:%d", clientName, req.ClientId)
}
defer func() {
- glog.V(0).Infof("disconnect %v local subscriber %s clientId:%d", clientName, req.PathPrefix, req.ClientId)
+ log.V(3).Infof("disconnect %v local subscriber %s clientId:%d", clientName, req.PathPrefix, req.ClientId)
fs.deleteClient("local", clientName, req.ClientId, req.ClientEpoch)
fs.listenersCond.Broadcast() // nudges the subscribers that are waiting
}()
lastReadTime := log_buffer.NewMessagePosition(req.SinceNs, -2)
- glog.V(0).Infof(" + %v local subscribe %s from %+v clientId:%d", clientName, req.PathPrefix, lastReadTime, req.ClientId)
+ log.V(3).Infof(" + %v local subscribe %s from %+v clientId:%d", clientName, req.PathPrefix, lastReadTime, req.ClientId)
eachEventNotificationFn := fs.eachEventNotificationFn(req, stream, clientName)
@@ -153,10 +153,10 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
for {
// println("reading from persisted logs ...")
- glog.V(0).Infof("read on disk %v local subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
+ log.V(3).Infof("read on disk %v local subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
processedTsNs, isDone, readPersistedLogErr = fs.filer.ReadPersistedLogBuffer(lastReadTime, req.UntilNs, eachLogEntryFn)
if readPersistedLogErr != nil {
- glog.V(0).Infof("read on disk %v local subscribe %s from %+v: %v", clientName, req.PathPrefix, lastReadTime, readPersistedLogErr)
+ log.V(3).Infof("read on disk %v local subscribe %s from %+v: %v", clientName, req.PathPrefix, lastReadTime, readPersistedLogErr)
return fmt.Errorf("reading from persisted logs: %v", readPersistedLogErr)
}
if isDone {
@@ -172,7 +172,7 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
}
}
- glog.V(0).Infof("read in memory %v local subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
+ log.V(3).Infof("read in memory %v local subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
lastReadTime, isDone, readInMemoryLogErr = fs.filer.LocalMetaLogBuffer.LoopProcessLogData("localMeta:"+clientName, lastReadTime, req.UntilNs, func() bool {
@@ -197,7 +197,7 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
if readInMemoryLogErr == log_buffer.ResumeFromDiskError {
continue
}
- glog.Errorf("processed to %v: %v", lastReadTime, readInMemoryLogErr)
+ log.Errorf("processed to %v: %v", lastReadTime, readInMemoryLogErr)
if readInMemoryLogErr != log_buffer.ResumeError {
break
}
@@ -218,7 +218,7 @@ func eachLogEntryFn(eachEventNotificationFn func(dirPath string, eventNotificati
return func(logEntry *filer_pb.LogEntry) (bool, error) {
event := &filer_pb.SubscribeMetadataResponse{}
if err := proto.Unmarshal(logEntry.Data, event); err != nil {
- glog.Errorf("unexpected unmarshal filer_pb.SubscribeMetadataResponse: %v", err)
+ log.Errorf("unexpected unmarshal filer_pb.SubscribeMetadataResponse: %v", err)
return false, fmt.Errorf("unexpected unmarshal filer_pb.SubscribeMetadataResponse: %v", err)
}
@@ -301,7 +301,7 @@ func (fs *FilerServer) eachEventNotificationFn(req *filer_pb.SubscribeMetadataRe
}
// println("sending", dirPath, entryName)
if err := stream.Send(message); err != nil {
- glog.V(0).Infof("=> client %v: %+v", clientName, err)
+ log.V(3).Infof("=> client %v: %+v", clientName, err)
return err
}
filtered = 0
@@ -329,7 +329,7 @@ func matchByDirectory(dirPath string, directories []string) bool {
func (fs *FilerServer) addClient(prefix string, clientType string, clientAddress string, clientId int32, clientEpoch int32) (isReplacing, alreadyKnown bool, clientName string) {
clientName = clientType + "@" + clientAddress
- glog.V(0).Infof("+ %v listener %v clientId %v clientEpoch %v", prefix, clientName, clientId, clientEpoch)
+ log.V(3).Infof("+ %v listener %v clientId %v clientEpoch %v", prefix, clientName, clientId, clientEpoch)
if clientId != 0 {
fs.knownListenersLock.Lock()
defer fs.knownListenersLock.Unlock()
@@ -345,7 +345,7 @@ func (fs *FilerServer) addClient(prefix string, clientType string, clientAddress
}
func (fs *FilerServer) deleteClient(prefix string, clientName string, clientId int32, clientEpoch int32) {
- glog.V(0).Infof("- %v listener %v clientId %v clientEpoch %v", prefix, clientName, clientId, clientEpoch)
+ log.V(3).Infof("- %v listener %v clientId %v clientEpoch %v", prefix, clientName, clientId, clientEpoch)
if clientId != 0 {
fs.knownListenersLock.Lock()
defer fs.knownListenersLock.Unlock()
diff --git a/weed/server/filer_grpc_server_traverse_meta.go b/weed/server/filer_grpc_server_traverse_meta.go
index 4a924f065..b393fda5c 100644
--- a/weed/server/filer_grpc_server_traverse_meta.go
+++ b/weed/server/filer_grpc_server_traverse_meta.go
@@ -4,7 +4,7 @@ import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/filer"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/util"
"github.com/viant/ptrie"
@@ -12,7 +12,7 @@ import (
func (fs *FilerServer) TraverseBfsMetadata(req *filer_pb.TraverseBfsMetadataRequest, stream filer_pb.SeaweedFiler_TraverseBfsMetadataServer) error {
- glog.V(0).Infof("TraverseBfsMetadata %v", req)
+ log.V(3).Infof("TraverseBfsMetadata %v", req)
excludedTrie := ptrie.New[bool]()
for _, excluded := range req.ExcludedPrefixes {
diff --git a/weed/server/filer_server.go b/weed/server/filer_server.go
index 090c795fa..5fad64736 100644
--- a/weed/server/filer_server.go
+++ b/weed/server/filer_server.go
@@ -42,7 +42,7 @@ import (
_ "github.com/seaweedfs/seaweedfs/weed/filer/sqlite"
_ "github.com/seaweedfs/seaweedfs/weed/filer/tarantool"
_ "github.com/seaweedfs/seaweedfs/weed/filer/ydb"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/notification"
_ "github.com/seaweedfs/seaweedfs/weed/notification/aws_sqs"
_ "github.com/seaweedfs/seaweedfs/weed/notification/gocdk_pub_sub"
@@ -143,7 +143,7 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
option.Masters.RefreshBySrvIfAvailable()
if len(option.Masters.GetInstances()) == 0 {
- glog.Fatal("master list is required!")
+ log.Fatal("master list is required!")
}
if !util.LoadConfiguration("filer", false) {
@@ -153,15 +153,15 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
if os.IsNotExist(err) {
os.MkdirAll(option.DefaultLevelDbDir, 0755)
}
- glog.V(0).Infof("default to create filer store dir in %s", option.DefaultLevelDbDir)
+ log.V(3).Infof("default to create filer store dir in %s", option.DefaultLevelDbDir)
} else {
- glog.Warningf("skipping default store dir in %s", option.DefaultLevelDbDir)
+ log.Warningf("skipping default store dir in %s", option.DefaultLevelDbDir)
}
util.LoadConfiguration("notification", false)
v.SetDefault("filer.options.max_file_name_length", 255)
maxFilenameLength := v.GetUint32("filer.options.max_file_name_length")
- glog.V(0).Infof("max_file_name_length %d", maxFilenameLength)
+ log.V(3).Infof("max_file_name_length %d", maxFilenameLength)
fs.filer = filer.NewFiler(*option.Masters, fs.grpcDialOption, option.Host, option.FilerGroup, option.Collection, option.DefaultReplication, option.DataCenter, maxFilenameLength, func() {
if atomic.LoadInt64(&fs.listenersWaits) > 0 {
fs.listenersCond.Broadcast()
@@ -201,9 +201,9 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
existingNodes := fs.filer.ListExistingPeerUpdates(context.Background())
startFromTime := time.Now().Add(-filer.LogFlushInterval)
if isFresh {
- glog.V(0).Infof("%s bootstrap from peers %+v", option.Host, existingNodes)
+ log.V(3).Infof("%s bootstrap from peers %+v", option.Host, existingNodes)
if err := fs.filer.MaybeBootstrapFromOnePeer(option.Host, existingNodes, startFromTime); err != nil {
- glog.Fatalf("%s bootstrap from %+v: %v", option.Host, existingNodes, err)
+ log.Fatalf("%s bootstrap from %+v: %v", option.Host, existingNodes, err)
}
}
fs.filer.AggregateFromPeers(option.Host, existingNodes, startFromTime)
@@ -246,7 +246,7 @@ func (fs *FilerServer) checkWithMaster() {
}
func (fs *FilerServer) Reload() {
- glog.V(0).Infoln("Reload filer server...")
+ log.V(3).Infoln("Reload filer server...")
util.LoadConfiguration("security", false)
v := util.GetViper()
diff --git a/weed/server/filer_server_handlers.go b/weed/server/filer_server_handlers.go
index 1c5c89dcf..5cf15a2a3 100644
--- a/weed/server/filer_server_handlers.go
+++ b/weed/server/filer_server_handlers.go
@@ -11,7 +11,7 @@ import (
"time"
"github.com/seaweedfs/seaweedfs/weed/filer"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/security"
"github.com/seaweedfs/seaweedfs/weed/util"
@@ -97,7 +97,7 @@ func (fs *FilerServer) filerHandler(w http.ResponseWriter, r *http.Request) {
fs.inFlightDataLimitCond.L.Lock()
inFlightDataSize := atomic.LoadInt64(&fs.inFlightDataSize)
for fs.option.ConcurrentUploadLimit != 0 && inFlightDataSize > fs.option.ConcurrentUploadLimit {
- glog.V(4).Infof("wait because inflight data %d > %d", inFlightDataSize, fs.option.ConcurrentUploadLimit)
+ log.V(-1).Infof("wait because inflight data %d > %d", inFlightDataSize, fs.option.ConcurrentUploadLimit)
fs.inFlightDataLimitCond.Wait()
inFlightDataSize = atomic.LoadInt64(&fs.inFlightDataSize)
}
@@ -211,17 +211,17 @@ func (fs *FilerServer) maybeCheckJwtAuthorization(r *http.Request, isWrite bool)
tokenStr := security.GetJwt(r)
if tokenStr == "" {
- glog.V(1).Infof("missing jwt from %s", r.RemoteAddr)
+ log.V(2).Infof("missing jwt from %s", r.RemoteAddr)
return false
}
token, err := security.DecodeJwt(signingKey, tokenStr, &security.SeaweedFilerClaims{})
if err != nil {
- glog.V(1).Infof("jwt verification error from %s: %v", r.RemoteAddr, err)
+ log.V(2).Infof("jwt verification error from %s: %v", r.RemoteAddr, err)
return false
}
if !token.Valid {
- glog.V(1).Infof("jwt invalid from %s: %v", r.RemoteAddr, tokenStr)
+ log.V(2).Infof("jwt invalid from %s: %v", r.RemoteAddr, tokenStr)
return false
} else {
return true
@@ -231,7 +231,7 @@ func (fs *FilerServer) maybeCheckJwtAuthorization(r *http.Request, isWrite bool)
func (fs *FilerServer) filerHealthzHandler(w http.ResponseWriter, r *http.Request) {
w.Header().Set("Server", "SeaweedFS "+util.VERSION)
if _, err := fs.filer.Store.FindEntry(context.Background(), filer.TopicsDir); err != nil && err != filer_pb.ErrNotFound {
- glog.Warningf("filerHealthzHandler FindEntry: %+v", err)
+ log.Warningf("filerHealthzHandler FindEntry: %+v", err)
w.WriteHeader(http.StatusServiceUnavailable)
} else {
w.WriteHeader(http.StatusOK)
diff --git a/weed/server/filer_server_handlers_proxy.go b/weed/server/filer_server_handlers_proxy.go
index ca445ef9a..eff17f57d 100644
--- a/weed/server/filer_server_handlers_proxy.go
+++ b/weed/server/filer_server_handlers_proxy.go
@@ -1,7 +1,7 @@
package weed_server
import (
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/security"
util_http "github.com/seaweedfs/seaweedfs/weed/util/http"
"github.com/seaweedfs/seaweedfs/weed/util/mem"
@@ -34,7 +34,7 @@ func (fs *FilerServer) proxyToVolumeServer(w http.ResponseWriter, r *http.Reques
urlStrings, err := fs.filer.MasterClient.GetLookupFileIdFunction()(fileId)
if err != nil {
- glog.Errorf("locate %s: %v", fileId, err)
+ log.Errorf("locate %s: %v", fileId, err)
w.WriteHeader(http.StatusInternalServerError)
return
}
@@ -46,7 +46,7 @@ func (fs *FilerServer) proxyToVolumeServer(w http.ResponseWriter, r *http.Reques
proxyReq, err := http.NewRequest(r.Method, urlStrings[rand.IntN(len(urlStrings))], r.Body)
if err != nil {
- glog.Errorf("NewRequest %s: %v", urlStrings[0], err)
+ log.Errorf("NewRequest %s: %v", urlStrings[0], err)
w.WriteHeader(http.StatusInternalServerError)
return
}
@@ -63,7 +63,7 @@ func (fs *FilerServer) proxyToVolumeServer(w http.ResponseWriter, r *http.Reques
proxyResponse, postErr := util_http.GetGlobalHttpClient().Do(proxyReq)
if postErr != nil {
- glog.Errorf("post to filer: %v", postErr)
+ log.Errorf("post to filer: %v", postErr)
w.WriteHeader(http.StatusInternalServerError)
return
}
diff --git a/weed/server/filer_server_handlers_read.go b/weed/server/filer_server_handlers_read.go
index 12371a8f6..acd3b6906 100644
--- a/weed/server/filer_server_handlers_read.go
+++ b/weed/server/filer_server_handlers_read.go
@@ -21,7 +21,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/util/mem"
"github.com/seaweedfs/seaweedfs/weed/filer"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/images"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/stats"
@@ -103,11 +103,11 @@ func (fs *FilerServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
return
}
if err == filer_pb.ErrNotFound {
- glog.V(2).Infof("Not found %s: %v", path, err)
+ log.V(1).Infof("Not found %s: %v", path, err)
stats.FilerHandlerCounter.WithLabelValues(stats.ErrorReadNotFound).Inc()
w.WriteHeader(http.StatusNotFound)
} else {
- glog.Errorf("Internal %s: %v", path, err)
+ log.Errorf("Internal %s: %v", path, err)
stats.FilerHandlerCounter.WithLabelValues(stats.ErrorReadInternal).Inc()
w.WriteHeader(http.StatusInternalServerError)
}
@@ -244,7 +244,7 @@ func (fs *FilerServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
defer mem.Free(data)
err := filer.ReadAll(data, fs.filer.MasterClient, entry.GetChunks())
if err != nil {
- glog.Errorf("failed to read %s: %v", path, err)
+ log.Errorf("failed to read %s: %v", path, err)
w.WriteHeader(http.StatusInternalServerError)
return
}
@@ -260,7 +260,7 @@ func (fs *FilerServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
_, err := writer.Write(entry.Content[offset : offset+size])
if err != nil {
stats.FilerHandlerCounter.WithLabelValues(stats.ErrorWriteEntry).Inc()
- glog.Errorf("failed to write entry content: %v", err)
+ log.Errorf("failed to write entry content: %v", err)
}
return err
}, nil
@@ -273,7 +273,7 @@ func (fs *FilerServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
Name: name,
}); err != nil {
stats.FilerHandlerCounter.WithLabelValues(stats.ErrorReadCache).Inc()
- glog.Errorf("CacheRemoteObjectToLocalCluster %s: %v", entry.FullPath, err)
+ log.Errorf("CacheRemoteObjectToLocalCluster %s: %v", entry.FullPath, err)
return nil, fmt.Errorf("cache %s: %v", entry.FullPath, err)
} else {
chunks = resp.Entry.GetChunks()
@@ -283,14 +283,14 @@ func (fs *FilerServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
streamFn, err := filer.PrepareStreamContentWithThrottler(fs.filer.MasterClient, fs.maybeGetVolumeReadJwtAuthorizationToken, chunks, offset, size, fs.option.DownloadMaxBytesPs)
if err != nil {
stats.FilerHandlerCounter.WithLabelValues(stats.ErrorReadStream).Inc()
- glog.Errorf("failed to prepare stream content %s: %v", r.URL, err)
+ log.Errorf("failed to prepare stream content %s: %v", r.URL, err)
return nil, err
}
return func(writer io.Writer) error {
err := streamFn(writer)
if err != nil {
stats.FilerHandlerCounter.WithLabelValues(stats.ErrorReadStream).Inc()
- glog.Errorf("failed to stream content %s: %v", r.URL, err)
+ log.Errorf("failed to stream content %s: %v", r.URL, err)
}
return err
}, nil
diff --git a/weed/server/filer_server_handlers_read_dir.go b/weed/server/filer_server_handlers_read_dir.go
index 56f0f9cb4..1f7aed16f 100644
--- a/weed/server/filer_server_handlers_read_dir.go
+++ b/weed/server/filer_server_handlers_read_dir.go
@@ -7,7 +7,7 @@ import (
"strconv"
"strings"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
ui "github.com/seaweedfs/seaweedfs/weed/server/filer_ui"
"github.com/seaweedfs/seaweedfs/weed/stats"
"github.com/seaweedfs/seaweedfs/weed/util"
@@ -43,7 +43,7 @@ func (fs *FilerServer) listDirectoryHandler(w http.ResponseWriter, r *http.Reque
entries, shouldDisplayLoadMore, err := fs.filer.ListDirectoryEntries(context.Background(), util.FullPath(path), lastFileName, false, int64(limit), "", namePattern, namePatternExclude)
if err != nil {
- glog.V(0).Infof("listDirectory %s %s %d: %s", path, lastFileName, limit, err)
+ log.V(3).Infof("listDirectory %s %s %d: %s", path, lastFileName, limit, err)
w.WriteHeader(http.StatusNotFound)
return
}
@@ -58,7 +58,7 @@ func (fs *FilerServer) listDirectoryHandler(w http.ResponseWriter, r *http.Reque
emptyFolder = false
}
- glog.V(4).Infof("listDirectory %s, last file %s, limit %d: %d items", path, lastFileName, limit, len(entries))
+ log.V(-1).Infof("listDirectory %s, last file %s, limit %d: %d items", path, lastFileName, limit, len(entries))
if r.Header.Get("Accept") == "application/json" {
writeJsonQuiet(w, r, http.StatusOK, struct {
@@ -103,7 +103,7 @@ func (fs *FilerServer) listDirectoryHandler(w http.ResponseWriter, r *http.Reque
fs.option.ShowUIDirectoryDelete,
})
if err != nil {
- glog.V(0).Infof("Template Execute Error: %v", err)
+ log.V(3).Infof("Template Execute Error: %v", err)
}
}
diff --git a/weed/server/filer_server_handlers_tagging.go b/weed/server/filer_server_handlers_tagging.go
index 80ea09d53..f7032837f 100644
--- a/weed/server/filer_server_handlers_tagging.go
+++ b/weed/server/filer_server_handlers_tagging.go
@@ -5,7 +5,7 @@ import (
"net/http"
"strings"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/storage/needle"
"github.com/seaweedfs/seaweedfs/weed/util"
)
@@ -44,7 +44,7 @@ func (fs *FilerServer) PutTaggingHandler(w http.ResponseWriter, r *http.Request)
}
if dbErr := fs.filer.CreateEntry(ctx, existingEntry, false, false, nil, false, fs.filer.MaxFilenameLength); dbErr != nil {
- glog.V(0).Infof("failing to update %s tagging : %v", path, dbErr)
+ log.V(3).Infof("failing to update %s tagging : %v", path, dbErr)
writeJsonError(w, r, http.StatusInternalServerError, dbErr)
return
}
@@ -110,7 +110,7 @@ func (fs *FilerServer) DeleteTaggingHandler(w http.ResponseWriter, r *http.Reque
}
if dbErr := fs.filer.CreateEntry(ctx, existingEntry, false, false, nil, false, fs.filer.MaxFilenameLength); dbErr != nil {
- glog.V(0).Infof("failing to delete %s tagging : %v", path, dbErr)
+ log.V(3).Infof("failing to delete %s tagging : %v", path, dbErr)
writeJsonError(w, r, http.StatusInternalServerError, dbErr)
return
}
diff --git a/weed/server/filer_server_handlers_write.go b/weed/server/filer_server_handlers_write.go
index 82880c2ac..df3b978f1 100644
--- a/weed/server/filer_server_handlers_write.go
+++ b/weed/server/filer_server_handlers_write.go
@@ -11,7 +11,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/s3api/s3_constants"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/security"
@@ -46,7 +46,7 @@ func (fs *FilerServer) assignNewFileInfo(so *operation.StorageOption) (fileId, u
assignResult, ae := operation.Assign(fs.filer.GetMaster, fs.grpcDialOption, ar, altRequest)
if ae != nil {
- glog.Errorf("failing to assign a file id: %v", ae)
+ log.Errorf("failing to assign a file id: %v", ae)
err = ae
return
}
@@ -93,14 +93,14 @@ func (fs *FilerServer) PostHandler(w http.ResponseWriter, r *http.Request, conte
if err == ErrReadOnly {
w.WriteHeader(http.StatusInsufficientStorage)
} else {
- glog.V(1).Infoln("post", r.RequestURI, ":", err.Error())
+ log.V(2).Infoln("post", r.RequestURI, ":", err.Error())
w.WriteHeader(http.StatusInternalServerError)
}
return
}
if util.FullPath(r.URL.Path).IsLongerFileName(so.MaxFileNameLength) {
- glog.V(1).Infoln("post", r.RequestURI, ": ", "entry name too long")
+ log.V(2).Infoln("post", r.RequestURI, ": ", "entry name too long")
w.WriteHeader(http.StatusRequestURITooLong)
return
}
@@ -128,7 +128,7 @@ func (fs *FilerServer) move(ctx context.Context, w http.ResponseWriter, r *http.
src := r.URL.Query().Get("mv.from")
dst := r.URL.Path
- glog.V(2).Infof("FilerServer.move %v to %v", src, dst)
+ log.V(1).Infof("FilerServer.move %v to %v", src, dst)
var err error
if src, err = clearName(src); err != nil {
@@ -232,7 +232,7 @@ func (fs *FilerServer) DeleteHandler(w http.ResponseWriter, r *http.Request) {
err = fs.filer.DeleteEntryMetaAndData(context.Background(), util.FullPath(objectPath), isRecursive, ignoreRecursiveError, !skipChunkDeletion, false, nil, 0)
if err != nil && err != filer_pb.ErrNotFound {
- glog.V(1).Infoln("deleting", objectPath, ":", err.Error())
+ log.V(2).Infoln("deleting", objectPath, ":", err.Error())
writeJsonError(w, r, http.StatusInternalServerError, err)
return
}
@@ -261,7 +261,7 @@ func (fs *FilerServer) detectStorageOption(requestURI, qCollection, qReplication
if ttlSeconds == 0 {
ttl, err := needle.ReadTTL(rule.GetTtl())
if err != nil {
- glog.Errorf("fail to parse %s ttl setting %s: %v", rule.LocationPrefix, rule.Ttl, err)
+ log.Errorf("fail to parse %s ttl setting %s: %v", rule.LocationPrefix, rule.Ttl, err)
}
ttlSeconds = int32(ttl.Minutes()) * 60
}
@@ -284,7 +284,7 @@ func (fs *FilerServer) detectStorageOption0(requestURI, qCollection, qReplicatio
ttl, err := needle.ReadTTL(qTtl)
if err != nil {
- glog.Errorf("fail to parse ttl %s: %v", qTtl, err)
+ log.Errorf("fail to parse ttl %s: %v", qTtl, err)
}
so, err := fs.detectStorageOption(requestURI, qCollection, qReplication, int32(ttl.Minutes())*60, diskType, dataCenter, rack, dataNode)
diff --git a/weed/server/filer_server_handlers_write_autochunk.go b/weed/server/filer_server_handlers_write_autochunk.go
index b0af7be4b..ee8cf4f52 100644
--- a/weed/server/filer_server_handlers_write_autochunk.go
+++ b/weed/server/filer_server_handlers_write_autochunk.go
@@ -14,7 +14,7 @@ import (
"time"
"github.com/seaweedfs/seaweedfs/weed/filer"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/s3api/s3_constants"
@@ -239,7 +239,7 @@ func (fs *FilerServer) saveMetaData(ctx context.Context, r *http.Request, fileNa
}
mode, err := strconv.ParseUint(modeStr, 8, 32)
if err != nil {
- glog.Errorf("Invalid mode format: %s, use 0660 by default", modeStr)
+ log.Errorf("Invalid mode format: %s, use 0660 by default", modeStr)
mode = 0660
}
@@ -256,7 +256,7 @@ func (fs *FilerServer) saveMetaData(ctx context.Context, r *http.Request, fileNa
if isAppend || isOffsetWrite {
existingEntry, findErr := fs.filer.FindEntry(ctx, util.FullPath(path))
if findErr != nil && findErr != filer_pb.ErrNotFound {
- glog.V(0).Infof("failing to find %s: %v", path, findErr)
+ log.V(3).Infof("failing to find %s: %v", path, findErr)
}
entry = existingEntry
}
@@ -279,7 +279,7 @@ func (fs *FilerServer) saveMetaData(ctx context.Context, r *http.Request, fileNa
}
} else {
- glog.V(4).Infoln("saving", path)
+ log.V(-1).Infoln("saving", path)
newChunks = fileChunks
entry = &filer.Entry{
FullPath: util.FullPath(path),
@@ -301,14 +301,14 @@ func (fs *FilerServer) saveMetaData(ctx context.Context, r *http.Request, fileNa
// maybe concatenate small chunks into one whole chunk
mergedChunks, replyerr = fs.maybeMergeChunks(so, newChunks)
if replyerr != nil {
- glog.V(0).Infof("merge chunks %s: %v", r.RequestURI, replyerr)
+ log.V(3).Infof("merge chunks %s: %v", r.RequestURI, replyerr)
mergedChunks = newChunks
}
// maybe compact entry chunks
mergedChunks, replyerr = filer.MaybeManifestize(fs.saveAsChunk(so), mergedChunks)
if replyerr != nil {
- glog.V(0).Infof("manifestize %s: %v", r.RequestURI, replyerr)
+ log.V(3).Infof("manifestize %s: %v", r.RequestURI, replyerr)
return
}
entry.Chunks = mergedChunks
@@ -343,7 +343,7 @@ func (fs *FilerServer) saveMetaData(ctx context.Context, r *http.Request, fileNa
if dbErr != nil {
replyerr = dbErr
filerResult.Error = dbErr.Error()
- glog.V(0).Infof("failing to write %s to filer server : %v", path, dbErr)
+ log.V(3).Infof("failing to write %s to filer server : %v", path, dbErr)
}
return filerResult, replyerr
}
@@ -403,7 +403,7 @@ func (fs *FilerServer) mkdir(ctx context.Context, w http.ResponseWriter, r *http
}
mode, err := strconv.ParseUint(modeStr, 8, 32)
if err != nil {
- glog.Errorf("Invalid mode format: %s, use 0660 by default", modeStr)
+ log.Errorf("Invalid mode format: %s, use 0660 by default", modeStr)
mode = 0660
}
@@ -419,7 +419,7 @@ func (fs *FilerServer) mkdir(ctx context.Context, w http.ResponseWriter, r *http
return
}
- glog.V(4).Infoln("mkdir", path)
+ log.V(-1).Infoln("mkdir", path)
entry := &filer.Entry{
FullPath: util.FullPath(path),
Attr: filer.Attr{
@@ -439,7 +439,7 @@ func (fs *FilerServer) mkdir(ctx context.Context, w http.ResponseWriter, r *http
if dbErr := fs.filer.CreateEntry(ctx, entry, false, false, nil, false, so.MaxFileNameLength); dbErr != nil {
replyerr = dbErr
filerResult.Error = dbErr.Error()
- glog.V(0).Infof("failing to create dir %s on filer server : %v", path, dbErr)
+ log.V(3).Infof("failing to create dir %s on filer server : %v", path, dbErr)
}
return filerResult, replyerr
}
diff --git a/weed/server/filer_server_handlers_write_cipher.go b/weed/server/filer_server_handlers_write_cipher.go
index 9c1628749..0de6cfb0f 100644
--- a/weed/server/filer_server_handlers_write_cipher.go
+++ b/weed/server/filer_server_handlers_write_cipher.go
@@ -9,7 +9,7 @@ import (
"time"
"github.com/seaweedfs/seaweedfs/weed/filer"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/storage/needle"
@@ -25,7 +25,7 @@ func (fs *FilerServer) encrypt(ctx context.Context, w http.ResponseWriter, r *ht
return nil, fmt.Errorf("fail to allocate volume for %s, collection:%s, datacenter:%s", r.URL.Path, so.Collection, so.DataCenter)
}
- glog.V(4).Infof("write %s to %v", r.URL.Path, urlLocation)
+ log.V(-1).Infof("write %s to %v", r.URL.Path, urlLocation)
// Note: encrypt(gzip(data)), encrypt data first, then gzip
diff --git a/weed/server/filer_server_handlers_write_merge.go b/weed/server/filer_server_handlers_write_merge.go
index 2110f485a..73bb06351 100644
--- a/weed/server/filer_server_handlers_write_merge.go
+++ b/weed/server/filer_server_handlers_write_merge.go
@@ -2,7 +2,7 @@ package weed_server
import (
"github.com/seaweedfs/seaweedfs/weed/filer"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/stats"
@@ -56,7 +56,7 @@ func (fs *FilerServer) mergeChunks(so *operation.StorageOption, inputChunks []*f
garbage, err := filer.MinusChunks(fs.lookupFileId, inputChunks, mergedChunks)
if err != nil {
- glog.Errorf("Failed to resolve old entry chunks when delete old entry chunks. new: %s, old: %s",
+ log.Errorf("Failed to resolve old entry chunks when delete old entry chunks. new: %s, old: %s",
mergedChunks, inputChunks)
return mergedChunks, err
}
diff --git a/weed/server/filer_server_handlers_write_upload.go b/weed/server/filer_server_handlers_write_upload.go
index e34fe27e6..0c3ef9418 100644
--- a/weed/server/filer_server_handlers_write_upload.go
+++ b/weed/server/filer_server_handlers_write_upload.go
@@ -13,7 +13,7 @@ import (
"slices"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/security"
@@ -130,7 +130,7 @@ func (fs *FilerServer) uploadReaderToChunks(reader io.Reader, startOffset int64,
fileChunksSize := len(fileChunks) + len(chunks)
for _, chunk := range chunks {
fileChunks = append(fileChunks, chunk)
- glog.V(4).Infof("uploaded %s chunk %d to %s [%d,%d)", fileName, fileChunksSize, chunk.FileId, offset, offset+int64(chunk.Size))
+ log.V(-1).Infof("uploaded %s chunk %d to %s [%d,%d)", fileName, fileChunksSize, chunk.FileId, offset, offset+int64(chunk.Size))
}
fileChunksLock.Unlock()
}
@@ -148,9 +148,9 @@ func (fs *FilerServer) uploadReaderToChunks(reader io.Reader, startOffset int64,
wg.Wait()
if uploadErr != nil {
- glog.V(0).Infof("upload file %s error: %v", fileName, uploadErr)
+ log.V(3).Infof("upload file %s error: %v", fileName, uploadErr)
for _, chunk := range fileChunks {
- glog.V(4).Infof("purging failed uploaded %s chunk %s [%d,%d)", fileName, chunk.FileId, chunk.Offset, chunk.Offset+int64(chunk.Size))
+ log.V(-1).Infof("purging failed uploaded %s chunk %s [%d,%d)", fileName, chunk.FileId, chunk.Offset, chunk.Offset+int64(chunk.Size))
}
fs.filer.DeleteUncommittedChunks(fileChunks)
return nil, md5Hash, 0, uploadErr, nil
@@ -205,14 +205,14 @@ func (fs *FilerServer) dataToChunk(fileName, contentType string, data []byte, ch
// assign one file id for one chunk
fileId, urlLocation, auth, uploadErr = fs.assignNewFileInfo(so)
if uploadErr != nil {
- glog.V(4).Infof("retry later due to assign error: %v", uploadErr)
+ log.V(-1).Infof("retry later due to assign error: %v", uploadErr)
stats.FilerHandlerCounter.WithLabelValues(stats.ChunkAssignRetry).Inc()
return uploadErr
}
// upload the chunk to the volume server
uploadResult, uploadErr, _ = fs.doUpload(urlLocation, dataReader, fileName, contentType, nil, auth)
if uploadErr != nil {
- glog.V(4).Infof("retry later due to upload error: %v", uploadErr)
+ log.V(-1).Infof("retry later due to upload error: %v", uploadErr)
stats.FilerHandlerCounter.WithLabelValues(stats.ChunkDoUploadRetry).Inc()
fid, _ := filer_pb.ToFileIdObject(fileId)
fileChunk := filer_pb.FileChunk{
@@ -226,7 +226,7 @@ func (fs *FilerServer) dataToChunk(fileName, contentType string, data []byte, ch
return nil
})
if err != nil {
- glog.Errorf("upload error: %v", err)
+ log.Errorf("upload error: %v", err)
return failedFileChunks, err
}
diff --git a/weed/server/master_grpc_server.go b/weed/server/master_grpc_server.go
index dcf279e1d..30a4dfd88 100644
--- a/weed/server/master_grpc_server.go
+++ b/weed/server/master_grpc_server.go
@@ -19,7 +19,7 @@ import (
"github.com/seaweedfs/raft"
"google.golang.org/grpc/peer"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
"github.com/seaweedfs/seaweedfs/weed/storage/needle"
"github.com/seaweedfs/seaweedfs/weed/topology"
@@ -39,7 +39,7 @@ func (ms *MasterServer) RegisterUuids(heartbeat *master_pb.Heartbeat) (duplicate
index := sort.SearchStrings(v, id)
if index < len(v) && v[index] == id {
duplicated_uuids = append(duplicated_uuids, id)
- glog.Errorf("directory of %s on %s has been loaded", id, k)
+ log.Errorf("directory of %s on %s has been loaded", id, k)
}
}
}
@@ -48,7 +48,7 @@ func (ms *MasterServer) RegisterUuids(heartbeat *master_pb.Heartbeat) (duplicate
}
ms.Topo.UuidMap[key] = heartbeat.LocationUuids
- glog.V(0).Infof("found new uuid:%v %v , %v", key, heartbeat.LocationUuids, ms.Topo.UuidMap)
+ log.V(3).Infof("found new uuid:%v %v , %v", key, heartbeat.LocationUuids, ms.Topo.UuidMap)
return nil, nil
}
@@ -57,7 +57,7 @@ func (ms *MasterServer) UnRegisterUuids(ip string, port int) {
defer ms.Topo.UuidAccessLock.Unlock()
key := fmt.Sprintf("%s:%d", ip, port)
delete(ms.Topo.UuidMap, key)
- glog.V(0).Infof("remove volume server %v, online volume server: %v", key, ms.Topo.UuidMap)
+ log.V(3).Infof("remove volume server %v, online volume server: %v", key, ms.Topo.UuidMap)
}
func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServer) error {
@@ -67,7 +67,7 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
if dn != nil {
dn.Counter--
if dn.Counter > 0 {
- glog.V(0).Infof("disconnect phantom volume server %s:%d remaining %d", dn.Ip, dn.Port, dn.Counter)
+ log.V(3).Infof("disconnect phantom volume server %s:%d remaining %d", dn.Ip, dn.Port, dn.Counter)
return
}
@@ -87,7 +87,7 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
// if the volume server disconnects and reconnects quickly
// the unregister and register can race with each other
ms.Topo.UnRegisterDataNode(dn)
- glog.V(0).Infof("unregister disconnected volume server %s:%d", dn.Ip, dn.Port)
+ log.V(3).Infof("unregister disconnected volume server %s:%d", dn.Ip, dn.Port)
ms.UnRegisterUuids(dn.Ip, dn.Port)
if ms.Topo.IsLeader() && (len(message.DeletedVids) > 0 || len(message.DeletedEcVids) > 0) {
@@ -100,9 +100,9 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
heartbeat, err := stream.Recv()
if err != nil {
if dn != nil {
- glog.Warningf("SendHeartbeat.Recv server %s:%d : %v", dn.Ip, dn.Port, err)
+ log.Warningf("SendHeartbeat.Recv server %s:%d : %v", dn.Ip, dn.Port, err)
} else {
- glog.Warningf("SendHeartbeat.Recv: %v", err)
+ log.Warningf("SendHeartbeat.Recv: %v", err)
}
stats.MasterReceivedHeartbeatCounter.WithLabelValues("error").Inc()
return err
@@ -112,16 +112,16 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
// tell the volume servers about the leader
newLeader, err := ms.Topo.Leader()
if err != nil {
- glog.Warningf("SendHeartbeat find leader: %v", err)
+ log.Warningf("SendHeartbeat find leader: %v", err)
return err
}
if err := stream.Send(&master_pb.HeartbeatResponse{
Leader: string(newLeader),
}); err != nil {
if dn != nil {
- glog.Warningf("SendHeartbeat.Send response to %s:%d %v", dn.Ip, dn.Port, err)
+ log.Warningf("SendHeartbeat.Send response to %s:%d %v", dn.Ip, dn.Port, err)
} else {
- glog.Warningf("SendHeartbeat.Send response %v", err)
+ log.Warningf("SendHeartbeat.Send response %v", err)
}
return err
}
@@ -138,13 +138,13 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
dc := ms.Topo.GetOrCreateDataCenter(dcName)
rack := dc.GetOrCreateRack(rackName)
dn = rack.GetOrCreateDataNode(heartbeat.Ip, int(heartbeat.Port), int(heartbeat.GrpcPort), heartbeat.PublicUrl, heartbeat.MaxVolumeCounts)
- glog.V(0).Infof("added volume server %d: %v:%d %v", dn.Counter, heartbeat.GetIp(), heartbeat.GetPort(), heartbeat.LocationUuids)
+ log.V(3).Infof("added volume server %d: %v:%d %v", dn.Counter, heartbeat.GetIp(), heartbeat.GetPort(), heartbeat.LocationUuids)
uuidlist, err := ms.RegisterUuids(heartbeat)
if err != nil {
if stream_err := stream.Send(&master_pb.HeartbeatResponse{
DuplicatedUuids: uuidlist,
}); stream_err != nil {
- glog.Warningf("SendHeartbeat.Send DuplicatedDirectory response to %s:%d %v", dn.Ip, dn.Port, stream_err)
+ log.Warningf("SendHeartbeat.Send DuplicatedDirectory response to %s:%d %v", dn.Ip, dn.Port, stream_err)
return stream_err
}
return err
@@ -154,7 +154,7 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
VolumeSizeLimit: uint64(ms.option.VolumeSizeLimitMB) * 1024 * 1024,
Preallocate: ms.preallocateSize > 0,
}); err != nil {
- glog.Warningf("SendHeartbeat.Send volume size to %s:%d %v", dn.Ip, dn.Port, err)
+ log.Warningf("SendHeartbeat.Send volume size to %s:%d %v", dn.Ip, dn.Port, err)
return err
}
stats.MasterReceivedHeartbeatCounter.WithLabelValues("dataNode").Inc()
@@ -163,7 +163,7 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
dn.AdjustMaxVolumeCounts(heartbeat.MaxVolumeCounts)
- glog.V(4).Infof("master received heartbeat %s", heartbeat.String())
+ log.V(-1).Infof("master received heartbeat %s", heartbeat.String())
stats.MasterReceivedHeartbeatCounter.WithLabelValues("total").Inc()
message := &master_pb.VolumeLocation{
@@ -201,11 +201,11 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
newVolumes, deletedVolumes := ms.Topo.SyncDataNodeRegistration(heartbeat.Volumes, dn)
for _, v := range newVolumes {
- glog.V(0).Infof("master see new volume %d from %s", uint32(v.Id), dn.Url())
+ log.V(3).Infof("master see new volume %d from %s", uint32(v.Id), dn.Url())
message.NewVids = append(message.NewVids, uint32(v.Id))
}
for _, v := range deletedVolumes {
- glog.V(0).Infof("master see deleted volume %d from %s", uint32(v.Id), dn.Url())
+ log.V(3).Infof("master see deleted volume %d from %s", uint32(v.Id), dn.Url())
message.DeletedVids = append(message.DeletedVids, uint32(v.Id))
}
}
@@ -229,7 +229,7 @@ func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServ
if len(heartbeat.EcShards) > 0 || heartbeat.HasNoEcShards {
stats.MasterReceivedHeartbeatCounter.WithLabelValues("ecShards").Inc()
- glog.V(4).Infof("master received ec shards from %s: %+v", dn.Url(), heartbeat.EcShards)
+ log.V(-1).Infof("master received ec shards from %s: %+v", dn.Url(), heartbeat.EcShards)
newShards, deletedShards := ms.Topo.SyncDataNodeEcShards(heartbeat.EcShards, dn)
// broadcast the ec vid changes to master clients
@@ -299,7 +299,7 @@ func (ms *MasterServer) KeepConnected(stream master_pb.Seaweed_KeepConnectedServ
for {
_, err := stream.Recv()
if err != nil {
- glog.V(2).Infof("- client %v: %v", clientName, err)
+ log.V(1).Infof("- client %v: %v", clientName, err)
go func() {
// consume message chan to avoid deadlock, go routine exit when message chan is closed
for range messageChan {
@@ -318,7 +318,7 @@ func (ms *MasterServer) KeepConnected(stream master_pb.Seaweed_KeepConnectedServ
select {
case message := <-messageChan:
if err := stream.Send(message); err != nil {
- glog.V(0).Infof("=> client %v: %+v", clientName, message)
+ log.V(3).Infof("=> client %v: %+v", clientName, message)
return err
}
case <-ticker.C:
@@ -342,10 +342,10 @@ func (ms *MasterServer) broadcastToClients(message *master_pb.KeepConnectedRespo
for client, ch := range ms.clientChans {
select {
case ch <- message:
- glog.V(4).Infof("send message to %s", client)
+ log.V(-1).Infof("send message to %s", client)
default:
stats.MasterBroadcastToFullErrorCounter.Inc()
- glog.Errorf("broadcastToClients %s message full", client)
+ log.Errorf("broadcastToClients %s message full", client)
}
}
ms.clientChansLock.RUnlock()
@@ -354,7 +354,7 @@ func (ms *MasterServer) broadcastToClients(message *master_pb.KeepConnectedRespo
func (ms *MasterServer) informNewLeader(stream master_pb.Seaweed_KeepConnectedServer) error {
leader, err := ms.Topo.Leader()
if err != nil {
- glog.Errorf("topo leader: %v", err)
+ log.Errorf("topo leader: %v", err)
return raft.NotLeaderError
}
if err := stream.Send(&master_pb.KeepConnectedResponse{
@@ -369,7 +369,7 @@ func (ms *MasterServer) informNewLeader(stream master_pb.Seaweed_KeepConnectedSe
func (ms *MasterServer) addClient(filerGroup, clientType string, clientAddress pb.ServerAddress) (clientName string, messageChan chan *master_pb.KeepConnectedResponse) {
clientName = filerGroup + "." + clientType + "@" + string(clientAddress)
- glog.V(0).Infof("+ client %v", clientName)
+ log.V(3).Infof("+ client %v", clientName)
// we buffer this because otherwise we end up in a potential deadlock where
// the KeepConnected loop is no longer listening on this channel but we're
@@ -384,7 +384,7 @@ func (ms *MasterServer) addClient(filerGroup, clientType string, clientAddress p
}
func (ms *MasterServer) deleteClient(clientName string) {
- glog.V(0).Infof("- client %v", clientName)
+ log.V(3).Infof("- client %v", clientName)
ms.clientChansLock.Lock()
// close message chan, so that the KeepConnected go routine can exit
if clientChan, ok := ms.clientChans[clientName]; ok {
@@ -398,11 +398,11 @@ func findClientAddress(ctx context.Context, grpcPort uint32) string {
// fmt.Printf("FromContext %+v\n", ctx)
pr, ok := peer.FromContext(ctx)
if !ok {
- glog.Error("failed to get peer from ctx")
+ log.Error("failed to get peer from ctx")
return ""
}
if pr.Addr == net.Addr(nil) {
- glog.Error("failed to get peer address")
+ log.Error("failed to get peer address")
return ""
}
if grpcPort == 0 {
diff --git a/weed/server/master_grpc_server_admin.go b/weed/server/master_grpc_server_admin.go
index d8b63b5da..1a241c2bf 100644
--- a/weed/server/master_grpc_server_admin.go
+++ b/weed/server/master_grpc_server_admin.go
@@ -10,7 +10,7 @@ import (
"github.com/seaweedfs/raft"
"github.com/seaweedfs/seaweedfs/weed/cluster"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb"
@@ -90,7 +90,7 @@ func (locks *AdminLocks) isLocked(lockName string) (clientName string, message s
if !found {
return "", "", false
}
- glog.V(4).Infof("isLocked %v: %v", adminLock.lastClient, adminLock.lastMessage)
+ log.V(-1).Infof("isLocked %v: %v", adminLock.lastClient, adminLock.lastMessage)
return adminLock.lastClient, adminLock.lastMessage, adminLock.accessLockTime.Add(LockDuration).After(time.Now())
}
@@ -132,7 +132,7 @@ func (ms *MasterServer) LeaseAdminToken(ctx context.Context, req *master_pb.Leas
}
if lastClient, lastMessage, isLocked := ms.adminLocks.isLocked(req.LockName); isLocked {
- glog.V(4).Infof("LeaseAdminToken %v", lastClient)
+ log.V(-1).Infof("LeaseAdminToken %v", lastClient)
if req.PreviousToken != 0 && ms.adminLocks.isValidToken(req.LockName, time.Unix(0, req.PreviousLockTime), req.PreviousToken) {
// for renew
ts, token := ms.adminLocks.generateToken(req.LockName, req.ClientName)
diff --git a/weed/server/master_grpc_server_assign.go b/weed/server/master_grpc_server_assign.go
index 4820de6a2..6f050af18 100644
--- a/weed/server/master_grpc_server_assign.go
+++ b/weed/server/master_grpc_server_assign.go
@@ -3,7 +3,7 @@ package weed_server
import (
"context"
"fmt"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/stats"
"strings"
"time"
@@ -22,16 +22,16 @@ func (ms *MasterServer) StreamAssign(server master_pb.Seaweed_StreamAssignServer
for {
req, err := server.Recv()
if err != nil {
- glog.Errorf("StreamAssign failed to receive: %v", err)
+ log.Errorf("StreamAssign failed to receive: %v", err)
return err
}
resp, err := ms.Assign(context.Background(), req)
if err != nil {
- glog.Errorf("StreamAssign failed to assign: %v", err)
+ log.Errorf("StreamAssign failed to assign: %v", err)
return err
}
if err = server.Send(resp); err != nil {
- glog.Errorf("StreamAssign failed to send: %v", err)
+ log.Errorf("StreamAssign failed to send: %v", err)
return err
}
}
@@ -98,7 +98,7 @@ func (ms *MasterServer) Assign(ctx context.Context, req *master_pb.AssignRequest
}
}
if err != nil {
- glog.V(1).Infof("assign %v %v: %v", req, option.String(), err)
+ log.V(2).Infof("assign %v %v: %v", req, option.String(), err)
stats.MasterPickForWriteErrorCounter.Inc()
lastErr = err
if (req.DataCenter != "" || req.Rack != "") && strings.Contains(err.Error(), topology.NoWritableVolumes) {
@@ -134,7 +134,7 @@ func (ms *MasterServer) Assign(ctx context.Context, req *master_pb.AssignRequest
}, nil
}
if lastErr != nil {
- glog.V(0).Infof("assign %v %v: %v", req, option.String(), lastErr)
+ log.V(3).Infof("assign %v %v: %v", req, option.String(), lastErr)
}
return nil, lastErr
}
diff --git a/weed/server/master_grpc_server_volume.go b/weed/server/master_grpc_server_volume.go
index 3a383e259..7388d35b2 100644
--- a/weed/server/master_grpc_server_volume.go
+++ b/weed/server/master_grpc_server_volume.go
@@ -15,7 +15,7 @@ import (
"github.com/seaweedfs/raft"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
"github.com/seaweedfs/seaweedfs/weed/security"
"github.com/seaweedfs/seaweedfs/weed/storage/needle"
@@ -28,12 +28,12 @@ const (
)
func (ms *MasterServer) DoAutomaticVolumeGrow(req *topology.VolumeGrowRequest) {
- glog.V(1).Infoln("starting automatic volume grow")
+ log.V(2).Infoln("starting automatic volume grow")
start := time.Now()
newVidLocations, err := ms.vg.AutomaticGrowByType(req.Option, ms.grpcDialOption, ms.Topo, req.Count)
- glog.V(1).Infoln("finished automatic volume grow, cost ", time.Now().Sub(start))
+ log.V(2).Infoln("finished automatic volume grow, cost ", time.Now().Sub(start))
if err != nil {
- glog.V(1).Infof("automatic volume grow failed: %+v", err)
+ log.V(2).Infof("automatic volume grow failed: %+v", err)
return
}
for _, newVidLocation := range newVidLocations {
@@ -77,7 +77,7 @@ func (ms *MasterServer) ProcessGrowRequest() {
_, err = ms.VolumeGrow(ctx, vgr)
}
if err != nil {
- glog.V(0).Infof("volume grow request failed: %+v", err)
+ log.V(3).Infof("volume grow request failed: %+v", err)
}
writableVolumes := vl.CloneWritableVolumes()
for dcId, racks := range dcs {
@@ -92,7 +92,7 @@ func (ms *MasterServer) ProcessGrowRequest() {
}
if _, err = ms.VolumeGrow(ctx, vgr); err != nil {
- glog.V(0).Infof("volume grow request for dc:%s rack:%s failed: %+v", dcId, rackId, err)
+ log.V(3).Infof("volume grow request for dc:%s rack:%s failed: %+v", dcId, rackId, err)
}
}
}
@@ -130,7 +130,7 @@ func (ms *MasterServer) ProcessGrowRequest() {
// not atomic but it's okay
if found || (!req.Force && !vl.ShouldGrowVolumes()) {
- glog.V(4).Infoln("discard volume grow request")
+ log.V(-1).Infoln("discard volume grow request")
time.Sleep(time.Millisecond * 211)
vl.DoneGrowRequest()
continue
@@ -138,7 +138,7 @@ func (ms *MasterServer) ProcessGrowRequest() {
filter.Store(req, nil)
// we have lock called inside vg
- glog.V(0).Infof("volume grow %+v", req)
+ log.V(3).Infof("volume grow %+v", req)
go func(req *topology.VolumeGrowRequest, vl *topology.VolumeLayout) {
ms.DoAutomaticVolumeGrow(req)
vl.DoneGrowRequest()
diff --git a/weed/server/master_server.go b/weed/server/master_server.go
index 8621708d2..8b323ecad 100644
--- a/weed/server/master_server.go
+++ b/weed/server/master_server.go
@@ -22,7 +22,7 @@ import (
"github.com/seaweedfs/raft"
"google.golang.org/grpc"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
"github.com/seaweedfs/seaweedfs/weed/security"
"github.com/seaweedfs/seaweedfs/weed/sequence"
@@ -125,11 +125,11 @@ func NewMasterServer(r *mux.Router, option *MasterOption, peers map[string]pb.Se
seq := ms.createSequencer(option)
if nil == seq {
- glog.Fatalf("create sequencer failed.")
+ log.Fatalf("create sequencer failed.")
}
ms.Topo = topology.NewTopology("topo", seq, uint64(ms.option.VolumeSizeLimitMB)*1024*1024, 5, replicationAsMin)
ms.vg = topology.NewDefaultVolumeGrowth()
- glog.V(0).Infoln("Volume Size Limit is", ms.option.VolumeSizeLimitMB, "MB")
+ log.V(3).Infoln("Volume Size Limit is", ms.option.VolumeSizeLimitMB, "MB")
ms.guard = security.NewGuard(append(ms.option.WhiteList, whiteList...), signingKey, expiresAfterSec, readSigningKey, readExpiresAfterSec)
@@ -178,10 +178,10 @@ func (ms *MasterServer) SetRaftServer(raftServer *RaftServer) {
if raftServer.raftServer != nil {
ms.Topo.RaftServer = raftServer.raftServer
ms.Topo.RaftServer.AddEventListener(raft.LeaderChangeEventType, func(e raft.Event) {
- glog.V(0).Infof("leader change event: %+v => %+v", e.PrevValue(), e.Value())
+ log.V(3).Infof("leader change event: %+v => %+v", e.PrevValue(), e.Value())
stats.MasterLeaderChangeCounter.WithLabelValues(fmt.Sprintf("%+v", e.Value())).Inc()
if ms.Topo.RaftServer.Leader() != "" {
- glog.V(0).Infof("[%s] %s becomes leader.", ms.Topo.RaftServer.Name(), ms.Topo.RaftServer.Leader())
+ log.V(3).Infof("[%s] %s becomes leader.", ms.Topo.RaftServer.Name(), ms.Topo.RaftServer.Leader())
ms.Topo.LastLeaderChangeTime = time.Now()
}
})
@@ -194,7 +194,7 @@ func (ms *MasterServer) SetRaftServer(raftServer *RaftServer) {
ms.Topo.RaftServerAccessLock.Unlock()
if ms.Topo.IsLeader() {
- glog.V(0).Infof("%s I am the leader!", raftServerName)
+ log.V(3).Infof("%s I am the leader!", raftServerName)
} else {
var raftServerLeader string
ms.Topo.RaftServerAccessLock.RLock()
@@ -206,7 +206,7 @@ func (ms *MasterServer) SetRaftServer(raftServer *RaftServer) {
raftServerLeader = string(raftServerLeaderAddr)
}
ms.Topo.RaftServerAccessLock.RUnlock()
- glog.V(0).Infof("%s %s - is the leader.", raftServerName, raftServerLeader)
+ log.V(3).Infof("%s %s - is the leader.", raftServerName, raftServerLeader)
}
}
@@ -233,7 +233,7 @@ func (ms *MasterServer) proxyToLeader(f http.HandlerFunc) http.HandlerFunc {
}
// proxy to leader
- glog.V(4).Infoln("proxying to leader", raftServerLeader)
+ log.V(-1).Infoln("proxying to leader", raftServerLeader)
proxy := httputil.NewSingleHostReverseProxy(targetUrl)
director := proxy.Director
proxy.Director = func(req *http.Request) {
@@ -254,7 +254,7 @@ func (ms *MasterServer) startAdminScripts() {
if adminScripts == "" {
return
}
- glog.V(0).Infof("adminScripts: %v", adminScripts)
+ log.V(3).Infof("adminScripts: %v", adminScripts)
v.SetDefault("master.maintenance.sleep_minutes", 17)
sleepMinutes := v.GetInt("master.maintenance.sleep_minutes")
@@ -313,12 +313,12 @@ func processEachCmd(reg *regexp.Regexp, line string, commandEnv *shell.CommandEn
for _, c := range shell.Commands {
if c.Name() == cmd {
if c.HasTag(shell.ResourceHeavy) {
- glog.Warningf("%s is resource heavy and should not run on master", cmd)
+ log.Warningf("%s is resource heavy and should not run on master", cmd)
continue
}
- glog.V(0).Infof("executing: %s %v", cmd, args)
+ log.V(3).Infof("executing: %s %v", cmd, args)
if err := c.Do(args, commandEnv, os.Stdout); err != nil {
- glog.V(0).Infof("error: %v", err)
+ log.V(3).Infof("error: %v", err)
}
}
}
@@ -328,14 +328,14 @@ func (ms *MasterServer) createSequencer(option *MasterOption) sequence.Sequencer
var seq sequence.Sequencer
v := util.GetViper()
seqType := strings.ToLower(v.GetString(SequencerType))
- glog.V(1).Infof("[%s] : [%s]", SequencerType, seqType)
+ log.V(2).Infof("[%s] : [%s]", SequencerType, seqType)
switch strings.ToLower(seqType) {
case "snowflake":
var err error
snowflakeId := v.GetInt(SequencerSnowflakeId)
seq, err = sequence.NewSnowflakeSequencer(string(option.Master), snowflakeId)
if err != nil {
- glog.Error(err)
+ log.Error(err)
seq = nil
}
case "raft":
@@ -353,7 +353,7 @@ func (ms *MasterServer) OnPeerUpdate(update *master_pb.ClusterNodeUpdate, startF
if update.NodeType != cluster.MasterType || ms.Topo.HashicorpRaft == nil {
return
}
- glog.V(4).Infof("OnPeerUpdate: %+v", update)
+ log.V(-1).Infof("OnPeerUpdate: %+v", update)
peerAddress := pb.ServerAddress(update.Address)
peerName := string(peerAddress)
@@ -368,7 +368,7 @@ func (ms *MasterServer) OnPeerUpdate(update *master_pb.ClusterNodeUpdate, startF
}
}
if !raftServerFound {
- glog.V(0).Infof("adding new raft server: %s", peerName)
+ log.V(3).Infof("adding new raft server: %s", peerName)
ms.Topo.HashicorpRaft.AddVoter(
hashicorpRaft.ServerID(peerName),
hashicorpRaft.ServerAddress(peerAddress.ToGrpcAddress()), 0, 0)
@@ -378,7 +378,7 @@ func (ms *MasterServer) OnPeerUpdate(update *master_pb.ClusterNodeUpdate, startF
ctx, cancel := context.WithTimeout(context.TODO(), 15*time.Second)
defer cancel()
if _, err := client.Ping(ctx, &master_pb.PingRequest{Target: string(peerAddress), TargetType: cluster.MasterType}); err != nil {
- glog.V(0).Infof("master %s didn't respond to pings. remove raft server", peerName)
+ log.V(3).Infof("master %s didn't respond to pings. remove raft server", peerName)
if err := ms.MasterClient.WithClient(false, func(client master_pb.SeaweedClient) error {
_, err := client.RaftRemoveServer(context.Background(), &master_pb.RaftRemoveServerRequest{
Id: peerName,
@@ -386,11 +386,11 @@ func (ms *MasterServer) OnPeerUpdate(update *master_pb.ClusterNodeUpdate, startF
})
return err
}); err != nil {
- glog.Warningf("failed removing old raft server: %v", err)
+ log.Warningf("failed removing old raft server: %v", err)
return err
}
} else {
- glog.V(0).Infof("master %s successfully responded to ping", peerName)
+ log.V(3).Infof("master %s successfully responded to ping", peerName)
}
return nil
})
@@ -408,7 +408,7 @@ func (ms *MasterServer) Shutdown() {
}
func (ms *MasterServer) Reload() {
- glog.V(0).Infoln("Reload master server...")
+ log.V(3).Infoln("Reload master server...")
util.LoadConfiguration("security", false)
v := util.GetViper()
diff --git a/weed/server/master_server_handlers.go b/weed/server/master_server_handlers.go
index f49b04e8c..a6d0e3ff5 100644
--- a/weed/server/master_server_handlers.go
+++ b/weed/server/master_server_handlers.go
@@ -7,7 +7,7 @@ import (
"strings"
"time"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/security"
@@ -143,7 +143,7 @@ func (ms *MasterServer) dirAssignHandler(w http.ResponseWriter, r *http.Request)
for time.Now().Sub(startTime) < maxTimeout {
fid, count, dnList, shouldGrow, err := ms.Topo.PickForWrite(requestedCount, option, vl)
if shouldGrow && !vl.HasGrowRequest() {
- glog.V(0).Infof("dirAssign volume growth %v from %v", option.String(), r.RemoteAddr)
+ log.V(3).Infof("dirAssign volume growth %v from %v", option.String(), r.RemoteAddr)
if err != nil && ms.Topo.AvailableSpaceFor(option) <= 0 {
err = fmt.Errorf("%s and no free volumes left for %s", err.Error(), option.String())
}
diff --git a/weed/server/master_server_handlers_admin.go b/weed/server/master_server_handlers_admin.go
index fb0503e33..6c3806ce2 100644
--- a/weed/server/master_server_handlers_admin.go
+++ b/weed/server/master_server_handlers_admin.go
@@ -10,7 +10,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb"
"github.com/seaweedfs/seaweedfs/weed/storage/backend/memory_map"
@@ -61,12 +61,12 @@ func (ms *MasterServer) volumeVacuumHandler(w http.ResponseWriter, r *http.Reque
var err error
gcThreshold, err = strconv.ParseFloat(gcString, 32)
if err != nil {
- glog.V(0).Infof("garbageThreshold %s is not a valid float number: %v", gcString, err)
+ log.V(3).Infof("garbageThreshold %s is not a valid float number: %v", gcString, err)
writeJsonError(w, r, http.StatusNotAcceptable, fmt.Errorf("garbageThreshold %s is not a valid float number", gcString))
return
}
}
- // glog.Infoln("garbageThreshold =", gcThreshold)
+ // log.Infoln("garbageThreshold =", gcThreshold)
ms.Topo.Vacuum(ms.grpcDialOption, gcThreshold, ms.option.MaxParallelVacuumPerServer, 0, "", ms.preallocateSize, false)
ms.dirStatusHandler(w, r)
}
@@ -78,7 +78,7 @@ func (ms *MasterServer) volumeGrowHandler(w http.ResponseWriter, r *http.Request
writeJsonError(w, r, http.StatusNotAcceptable, err)
return
}
- glog.V(0).Infof("volumeGrowHandler received %v from %v", option.String(), r.RemoteAddr)
+ log.V(3).Infof("volumeGrowHandler received %v from %v", option.String(), r.RemoteAddr)
if count, err = strconv.ParseUint(r.FormValue("count"), 10, 32); err == nil {
replicaCount := int64(count * uint64(option.ReplicaPlacement.GetCopyCount()))
diff --git a/weed/server/raft_hashicorp.go b/weed/server/raft_hashicorp.go
index 0c6a72d6f..42fb20867 100644
--- a/weed/server/raft_hashicorp.go
+++ b/weed/server/raft_hashicorp.go
@@ -18,9 +18,9 @@ import (
"github.com/armon/go-metrics/prometheus"
"github.com/hashicorp/raft"
boltdb "github.com/hashicorp/raft-boltdb/v2"
- "github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/stats"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"google.golang.org/grpc"
)
@@ -76,7 +76,7 @@ func (s *RaftServer) monitorLeaderLoop(updatePeers bool) {
} else {
s.topo.BarrierReset()
}
- glog.V(0).Infof("is leader %+v change event: %+v => %+v", isLeader, prevLeader, leader)
+ log.V(3).Infof("is leader %+v change event: %+v => %+v", isLeader, prevLeader, leader)
prevLeader = leader
s.topo.LastLeaderChangeTime = time.Now()
}
@@ -97,18 +97,18 @@ func (s *RaftServer) updatePeers() {
if peerName == peerLeader || existsPeerName[peerName] {
continue
}
- glog.V(0).Infof("adding new peer: %s", peerName)
+ log.V(3).Infof("adding new peer: %s", peerName)
s.RaftHashicorp.AddVoter(
raft.ServerID(peerName), raft.ServerAddress(peer.ToGrpcAddress()), 0, 0)
}
for peer := range existsPeerName {
if _, found := s.peers[peer]; !found {
- glog.V(0).Infof("removing old peer: %s", peer)
+ log.V(3).Infof("removing old peer: %s", peer)
s.RaftHashicorp.RemoveServer(raft.ServerID(peer), 0, 0)
}
}
if _, found := s.peers[peerLeader]; !found {
- glog.V(0).Infof("removing old leader peer: %s", peerLeader)
+ log.V(3).Infof("removing old leader peer: %s", peerLeader)
s.RaftHashicorp.RemoveServer(raft.ServerID(peerLeader), 0, 0)
}
}
@@ -128,13 +128,13 @@ func NewHashicorpRaftServer(option *RaftServerOption) (*RaftServer, error) {
if c.LeaderLeaseTimeout > c.HeartbeatTimeout {
c.LeaderLeaseTimeout = c.HeartbeatTimeout
}
- if glog.V(4) {
+ if log.V(-1).Info != nil {
c.LogLevel = "Debug"
- } else if glog.V(2) {
+ } else if log.V(1).Info != nil {
c.LogLevel = "Info"
- } else if glog.V(1) {
+ } else if log.V(2).Info != nil {
c.LogLevel = "Warn"
- } else if glog.V(0) {
+ } else if log.V(3).Info != nil {
c.LogLevel = "Error"
}
@@ -181,7 +181,7 @@ func NewHashicorpRaftServer(option *RaftServerOption) (*RaftServer, error) {
// Need to get lock, in case all servers do this at the same time.
peerIdx := getPeerIdx(s.serverAddr, s.peers)
timeSleep := time.Duration(float64(c.LeaderLeaseTimeout) * (rand.Float64()*0.25 + 1) * float64(peerIdx))
- glog.V(0).Infof("Bootstrapping idx: %d sleep: %v new cluster: %+v", peerIdx, timeSleep, cfg)
+ log.V(3).Infof("Bootstrapping idx: %d sleep: %v new cluster: %+v", peerIdx, timeSleep, cfg)
time.Sleep(timeSleep)
f := s.RaftHashicorp.BootstrapCluster(cfg)
if err := f.Error(); err != nil {
@@ -194,17 +194,17 @@ func NewHashicorpRaftServer(option *RaftServerOption) (*RaftServer, error) {
go s.monitorLeaderLoop(updatePeers)
ticker := time.NewTicker(c.HeartbeatTimeout * 10)
- if glog.V(4) {
+ if log.V(-1).Info != nil {
go func() {
for {
select {
case <-ticker.C:
cfuture := s.RaftHashicorp.GetConfiguration()
if err = cfuture.Error(); err != nil {
- glog.Fatalf("error getting config: %s", err)
+ log.Fatalf("error getting config: %s", err)
}
configuration := cfuture.Configuration()
- glog.V(4).Infof("Showing peers known by %s:\n%+v", s.RaftHashicorp.String(), configuration.Servers)
+ log.V(-1).Infof("Showing peers known by %s:\n%+v", s.RaftHashicorp.String(), configuration.Servers)
}
}
}()
diff --git a/weed/server/raft_server.go b/weed/server/raft_server.go
index 4d2209dc0..382a5cd6a 100644
--- a/weed/server/raft_server.go
+++ b/weed/server/raft_server.go
@@ -17,8 +17,8 @@ import (
hashicorpRaft "github.com/hashicorp/raft"
"github.com/seaweedfs/raft"
- "github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/topology"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
)
type RaftServerOption struct {
@@ -55,7 +55,7 @@ func (s StateMachine) Save() ([]byte, error) {
state := topology.MaxVolumeIdCommand{
MaxVolumeId: s.topo.GetMaxVolumeId(),
}
- glog.V(1).Infof("Save raft state %+v", state)
+ log.V(2).Infof("Save raft state %+v", state)
return json.Marshal(state)
}
@@ -65,7 +65,7 @@ func (s StateMachine) Recovery(data []byte) error {
if err != nil {
return err
}
- glog.V(1).Infof("Recovery raft state %+v", state)
+ log.V(2).Infof("Recovery raft state %+v", state)
s.topo.UpAdjustMaxVolumeId(state.MaxVolumeId)
return nil
}
@@ -79,7 +79,7 @@ func (s *StateMachine) Apply(l *hashicorpRaft.Log) interface{} {
}
s.topo.UpAdjustMaxVolumeId(state.MaxVolumeId)
- glog.V(1).Infoln("max volume id", before, "==>", s.topo.GetMaxVolumeId())
+ log.V(2).Infoln("max volume id", before, "==>", s.topo.GetMaxVolumeId())
return nil
}
@@ -108,7 +108,7 @@ func NewRaftServer(option *RaftServerOption) (*RaftServer, error) {
topo: option.Topo,
}
- if glog.V(4) {
+ if log.V(-1).Info != nil {
raft.SetLogLevel(2)
}
@@ -116,7 +116,7 @@ func NewRaftServer(option *RaftServerOption) (*RaftServer, error) {
var err error
transporter := raft.NewGrpcTransporter(option.GrpcDialOption)
- glog.V(0).Infof("Starting RaftServer with %v", option.ServerAddr)
+ log.V(3).Infof("Starting RaftServer with %v", option.ServerAddr)
// always clear previous log to avoid server is promotable
os.RemoveAll(path.Join(s.dataDir, "log"))
@@ -132,7 +132,7 @@ func NewRaftServer(option *RaftServerOption) (*RaftServer, error) {
stateMachine := StateMachine{topo: option.Topo}
s.raftServer, err = raft.NewServer(string(s.serverAddr), s.dataDir, transporter, stateMachine, option.Topo, s.serverAddr.ToGrpcAddress())
if err != nil {
- glog.V(0).Infoln(err)
+ log.V(3).Infoln(err)
return nil, err
}
heartbeatInterval := time.Duration(float64(option.HeartbeatInterval) * (rand.Float64()*0.25 + 1))
@@ -155,17 +155,17 @@ func NewRaftServer(option *RaftServerOption) (*RaftServer, error) {
for existsPeerName := range s.raftServer.Peers() {
if existingPeer, found := s.peers[existsPeerName]; !found {
if err := s.raftServer.RemovePeer(existsPeerName); err != nil {
- glog.V(0).Infoln(err)
+ log.V(3).Infoln(err)
return nil, err
} else {
- glog.V(0).Infof("removing old peer: %s", existingPeer)
+ log.V(3).Infof("removing old peer: %s", existingPeer)
}
}
}
s.GrpcServer = raft.NewGrpcServer(s.raftServer)
- glog.V(0).Infof("current cluster leader: %v", s.raftServer.Leader())
+ log.V(3).Infof("current cluster leader: %v", s.raftServer.Leader())
return s, nil
}
@@ -187,13 +187,13 @@ func (s *RaftServer) Peers() (members []string) {
func (s *RaftServer) DoJoinCommand() {
- glog.V(0).Infoln("Initializing new cluster")
+ log.V(3).Infoln("Initializing new cluster")
if _, err := s.raftServer.Do(&raft.DefaultJoinCommand{
Name: s.raftServer.Name(),
ConnectionString: s.serverAddr.ToGrpcAddress(),
}); err != nil {
- glog.Errorf("fail to send join command: %v", err)
+ log.Errorf("fail to send join command: %v", err)
}
}
diff --git a/weed/server/raft_server_handlers.go b/weed/server/raft_server_handlers.go
index eba61569e..d815e117e 100644
--- a/weed/server/raft_server_handlers.go
+++ b/weed/server/raft_server_handlers.go
@@ -2,7 +2,7 @@ package weed_server
import (
"github.com/cenkalti/backoff/v4"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/storage/needle"
"net/http"
@@ -42,7 +42,7 @@ func (s *RaftServer) HealthzHandler(w http.ResponseWriter, r *http.Request) {
expBackoff.MaxElapsedTime = 5 * time.Second
isLocked, err := backoff.RetryWithData(s.topo.IsChildLocked, expBackoff)
if err != nil {
- glog.Errorf("HealthzHandler: %+v", err)
+ log.Errorf("HealthzHandler: %+v", err)
}
if isLocked {
w.WriteHeader(http.StatusLocked)
diff --git a/weed/server/volume_grpc_admin.go b/weed/server/volume_grpc_admin.go
index 2c5a538e7..57cafff18 100644
--- a/weed/server/volume_grpc_admin.go
+++ b/weed/server/volume_grpc_admin.go
@@ -14,7 +14,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
"github.com/seaweedfs/seaweedfs/weed/util"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb"
"github.com/seaweedfs/seaweedfs/weed/stats"
"github.com/seaweedfs/seaweedfs/weed/storage/needle"
@@ -29,9 +29,9 @@ func (vs *VolumeServer) DeleteCollection(ctx context.Context, req *volume_server
err := vs.store.DeleteCollection(req.Collection)
if err != nil {
- glog.Errorf("delete collection %s: %v", req.Collection, err)
+ log.Errorf("delete collection %s: %v", req.Collection, err)
} else {
- glog.V(2).Infof("delete collection %v", req)
+ log.V(1).Infof("delete collection %v", req)
}
return resp, err
@@ -55,9 +55,9 @@ func (vs *VolumeServer) AllocateVolume(ctx context.Context, req *volume_server_p
)
if err != nil {
- glog.Errorf("assign volume %v: %v", req, err)
+ log.Errorf("assign volume %v: %v", req, err)
} else {
- glog.V(2).Infof("assign volume %v", req)
+ log.V(1).Infof("assign volume %v", req)
}
return resp, err
@@ -71,9 +71,9 @@ func (vs *VolumeServer) VolumeMount(ctx context.Context, req *volume_server_pb.V
err := vs.store.MountVolume(needle.VolumeId(req.VolumeId))
if err != nil {
- glog.Errorf("volume mount %v: %v", req, err)
+ log.Errorf("volume mount %v: %v", req, err)
} else {
- glog.V(2).Infof("volume mount %v", req)
+ log.V(1).Infof("volume mount %v", req)
}
return resp, err
@@ -87,9 +87,9 @@ func (vs *VolumeServer) VolumeUnmount(ctx context.Context, req *volume_server_pb
err := vs.store.UnmountVolume(needle.VolumeId(req.VolumeId))
if err != nil {
- glog.Errorf("volume unmount %v: %v", req, err)
+ log.Errorf("volume unmount %v: %v", req, err)
} else {
- glog.V(2).Infof("volume unmount %v", req)
+ log.V(1).Infof("volume unmount %v", req)
}
return resp, err
@@ -103,9 +103,9 @@ func (vs *VolumeServer) VolumeDelete(ctx context.Context, req *volume_server_pb.
err := vs.store.DeleteVolume(needle.VolumeId(req.VolumeId), req.OnlyEmpty)
if err != nil {
- glog.Errorf("volume delete %v: %v", req, err)
+ log.Errorf("volume delete %v: %v", req, err)
} else {
- glog.V(2).Infof("volume delete %v", req)
+ log.V(1).Infof("volume delete %v", req)
}
return resp, err
@@ -124,21 +124,21 @@ func (vs *VolumeServer) VolumeConfigure(ctx context.Context, req *volume_server_
// unmount
if err := vs.store.UnmountVolume(needle.VolumeId(req.VolumeId)); err != nil {
- glog.Errorf("volume configure unmount %v: %v", req, err)
+ log.Errorf("volume configure unmount %v: %v", req, err)
resp.Error = fmt.Sprintf("volume configure unmount %v: %v", req, err)
return resp, nil
}
// modify the volume info file
if err := vs.store.ConfigureVolume(needle.VolumeId(req.VolumeId), req.Replication); err != nil {
- glog.Errorf("volume configure %v: %v", req, err)
+ log.Errorf("volume configure %v: %v", req, err)
resp.Error = fmt.Sprintf("volume configure %v: %v", req, err)
return resp, nil
}
// mount
if err := vs.store.MountVolume(needle.VolumeId(req.VolumeId)); err != nil {
- glog.Errorf("volume configure mount %v: %v", req, err)
+ log.Errorf("volume configure mount %v: %v", req, err)
resp.Error = fmt.Sprintf("volume configure mount %v: %v", req, err)
return resp, nil
}
@@ -167,9 +167,9 @@ func (vs *VolumeServer) VolumeMarkReadonly(ctx context.Context, req *volume_serv
err := vs.store.MarkVolumeReadonly(needle.VolumeId(req.VolumeId), req.GetPersist())
if err != nil {
- glog.Errorf("volume mark readonly %v: %v", req, err)
+ log.Errorf("volume mark readonly %v: %v", req, err)
} else {
- glog.V(2).Infof("volume mark readonly %v", req)
+ log.V(1).Infof("volume mark readonly %v", req)
}
// step 3: tell master from redirecting traffic here again, to prevent rare case 1.5
@@ -197,7 +197,7 @@ func (vs *VolumeServer) notifyMasterVolumeReadonly(v *storage.Volume, isReadOnly
}
return nil
}); grpcErr != nil {
- glog.V(0).Infof("connect to %s: %v", vs.GetMaster(context.Background()), grpcErr)
+ log.V(3).Infof("connect to %s: %v", vs.GetMaster(context.Background()), grpcErr)
return fmt.Errorf("grpc VolumeMarkReadonly with master %s: %v", vs.GetMaster(context.Background()), grpcErr)
}
return nil
@@ -215,9 +215,9 @@ func (vs *VolumeServer) VolumeMarkWritable(ctx context.Context, req *volume_serv
err := vs.store.MarkVolumeWritable(needle.VolumeId(req.VolumeId))
if err != nil {
- glog.Errorf("volume mark writable %v: %v", req, err)
+ log.Errorf("volume mark writable %v: %v", req, err)
} else {
- glog.V(2).Infof("volume mark writable %v", req)
+ log.V(1).Infof("volume mark writable %v", req)
}
// enable master to redirect traffic here
diff --git a/weed/server/volume_grpc_client_to_master.go b/weed/server/volume_grpc_client_to_master.go
index 2f9bc5965..1cba905b4 100644
--- a/weed/server/volume_grpc_client_to_master.go
+++ b/weed/server/volume_grpc_client_to_master.go
@@ -16,7 +16,7 @@ import (
"golang.org/x/net/context"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
"github.com/seaweedfs/seaweedfs/weed/util"
)
@@ -40,7 +40,7 @@ func (vs *VolumeServer) checkWithMaster() (err error) {
if err == nil {
return
} else {
- glog.V(0).Infof("checkWithMaster %s: %v", master, err)
+ log.V(3).Infof("checkWithMaster %s: %v", master, err)
}
}
time.Sleep(1790 * time.Millisecond)
@@ -49,7 +49,7 @@ func (vs *VolumeServer) checkWithMaster() (err error) {
func (vs *VolumeServer) heartbeat() {
- glog.V(0).Infof("Volume server start with seed master nodes: %v", vs.SeedMasterNodes)
+ log.V(3).Infof("Volume server start with seed master nodes: %v", vs.SeedMasterNodes)
vs.store.SetDataCenter(vs.dataCenter)
vs.store.SetRack(vs.rack)
@@ -68,7 +68,7 @@ func (vs *VolumeServer) heartbeat() {
vs.store.MasterAddress = master
newLeader, err = vs.doHeartbeat(master, grpcDialOption, time.Duration(vs.pulseSeconds)*time.Second)
if err != nil {
- glog.V(0).Infof("heartbeat to %s error: %v", master, err)
+ log.V(3).Infof("heartbeat to %s error: %v", master, err)
time.Sleep(time.Duration(vs.pulseSeconds) * time.Second)
newLeader = ""
vs.store.MasterAddress = ""
@@ -103,10 +103,10 @@ func (vs *VolumeServer) doHeartbeat(masterAddress pb.ServerAddress, grpcDialOpti
client := master_pb.NewSeaweedClient(grpcConnection)
stream, err := client.SendHeartbeat(ctx)
if err != nil {
- glog.V(0).Infof("SendHeartbeat to %s: %v", masterAddress, err)
+ log.V(3).Infof("SendHeartbeat to %s: %v", masterAddress, err)
return "", err
}
- glog.V(0).Infof("Heartbeat to: %v", masterAddress)
+ log.V(3).Infof("Heartbeat to: %v", masterAddress)
vs.currentMaster = masterAddress
doneChan := make(chan error, 1)
@@ -127,7 +127,7 @@ func (vs *VolumeServer) doHeartbeat(masterAddress pb.ServerAddress, grpcDialOpti
}
}
}
- glog.Errorf("Shut down Volume Server due to duplicate volume directories: %v", duplicateDir)
+ log.Errorf("Shut down Volume Server due to duplicate volume directories: %v", duplicateDir)
os.Exit(1)
}
volumeOptsChanged := false
@@ -142,13 +142,13 @@ func (vs *VolumeServer) doHeartbeat(masterAddress pb.ServerAddress, grpcDialOpti
if volumeOptsChanged {
if vs.store.MaybeAdjustVolumeMax() {
if err = stream.Send(vs.store.CollectHeartbeat()); err != nil {
- glog.V(0).Infof("Volume Server Failed to talk with master %s: %v", vs.currentMaster, err)
+ log.V(3).Infof("Volume Server Failed to talk with master %s: %v", vs.currentMaster, err)
return
}
}
}
if in.GetLeader() != "" && string(vs.currentMaster) != in.GetLeader() {
- glog.V(0).Infof("Volume Server found a new master newLeader: %v instead of %v", in.GetLeader(), vs.currentMaster)
+ log.V(3).Infof("Volume Server found a new master newLeader: %v instead of %v", in.GetLeader(), vs.currentMaster)
newLeader = pb.ServerAddress(in.GetLeader())
doneChan <- nil
return
@@ -157,12 +157,12 @@ func (vs *VolumeServer) doHeartbeat(masterAddress pb.ServerAddress, grpcDialOpti
}()
if err = stream.Send(vs.store.CollectHeartbeat()); err != nil {
- glog.V(0).Infof("Volume Server Failed to talk with master %s: %v", masterAddress, err)
+ log.V(3).Infof("Volume Server Failed to talk with master %s: %v", masterAddress, err)
return "", err
}
if err = stream.Send(vs.store.CollectErasureCodingHeartbeat()); err != nil {
- glog.V(0).Infof("Volume Server Failed to talk with master %s: %v", masterAddress, err)
+ log.V(3).Infof("Volume Server Failed to talk with master %s: %v", masterAddress, err)
return "", err
}
@@ -186,9 +186,9 @@ func (vs *VolumeServer) doHeartbeat(masterAddress pb.ServerAddress, grpcDialOpti
&volumeMessage,
},
}
- glog.V(0).Infof("volume server %s:%d adds volume %d", vs.store.Ip, vs.store.Port, volumeMessage.Id)
+ log.V(3).Infof("volume server %s:%d adds volume %d", vs.store.Ip, vs.store.Port, volumeMessage.Id)
if err = stream.Send(deltaBeat); err != nil {
- glog.V(0).Infof("Volume Server Failed to update to master %s: %v", masterAddress, err)
+ log.V(3).Infof("Volume Server Failed to update to master %s: %v", masterAddress, err)
return "", err
}
case ecShardMessage := <-vs.store.NewEcShardsChan:
@@ -201,10 +201,10 @@ func (vs *VolumeServer) doHeartbeat(masterAddress pb.ServerAddress, grpcDialOpti
&ecShardMessage,
},
}
- glog.V(0).Infof("volume server %s:%d adds ec shard %d:%d", vs.store.Ip, vs.store.Port, ecShardMessage.Id,
+ log.V(3).Infof("volume server %s:%d adds ec shard %d:%d", vs.store.Ip, vs.store.Port, ecShardMessage.Id,
erasure_coding.ShardBits(ecShardMessage.EcIndexBits).ShardIds())
if err = stream.Send(deltaBeat); err != nil {
- glog.V(0).Infof("Volume Server Failed to update to master %s: %v", masterAddress, err)
+ log.V(3).Infof("Volume Server Failed to update to master %s: %v", masterAddress, err)
return "", err
}
case volumeMessage := <-vs.store.DeletedVolumesChan:
@@ -217,9 +217,9 @@ func (vs *VolumeServer) doHeartbeat(masterAddress pb.ServerAddress, grpcDialOpti
&volumeMessage,
},
}
- glog.V(0).Infof("volume server %s:%d deletes volume %d", vs.store.Ip, vs.store.Port, volumeMessage.Id)
+ log.V(3).Infof("volume server %s:%d deletes volume %d", vs.store.Ip, vs.store.Port, volumeMessage.Id)
if err = stream.Send(deltaBeat); err != nil {
- glog.V(0).Infof("Volume Server Failed to update to master %s: %v", masterAddress, err)
+ log.V(3).Infof("Volume Server Failed to update to master %s: %v", masterAddress, err)
return "", err
}
case ecShardMessage := <-vs.store.DeletedEcShardsChan:
@@ -232,23 +232,23 @@ func (vs *VolumeServer) doHeartbeat(masterAddress pb.ServerAddress, grpcDialOpti
&ecShardMessage,
},
}
- glog.V(0).Infof("volume server %s:%d deletes ec shard %d:%d", vs.store.Ip, vs.store.Port, ecShardMessage.Id,
+ log.V(3).Infof("volume server %s:%d deletes ec shard %d:%d", vs.store.Ip, vs.store.Port, ecShardMessage.Id,
erasure_coding.ShardBits(ecShardMessage.EcIndexBits).ShardIds())
if err = stream.Send(deltaBeat); err != nil {
- glog.V(0).Infof("Volume Server Failed to update to master %s: %v", masterAddress, err)
+ log.V(3).Infof("Volume Server Failed to update to master %s: %v", masterAddress, err)
return "", err
}
case <-volumeTickChan.C:
- glog.V(4).Infof("volume server %s:%d heartbeat", vs.store.Ip, vs.store.Port)
+ log.V(-1).Infof("volume server %s:%d heartbeat", vs.store.Ip, vs.store.Port)
vs.store.MaybeAdjustVolumeMax()
if err = stream.Send(vs.store.CollectHeartbeat()); err != nil {
- glog.V(0).Infof("Volume Server Failed to talk with master %s: %v", masterAddress, err)
+ log.V(3).Infof("Volume Server Failed to talk with master %s: %v", masterAddress, err)
return "", err
}
case <-ecShardTickChan.C:
- glog.V(4).Infof("volume server %s:%d ec heartbeat", vs.store.Ip, vs.store.Port)
+ log.V(-1).Infof("volume server %s:%d ec heartbeat", vs.store.Ip, vs.store.Port)
if err = stream.Send(vs.store.CollectErasureCodingHeartbeat()); err != nil {
- glog.V(0).Infof("Volume Server Failed to talk with master %s: %v", masterAddress, err)
+ log.V(3).Infof("Volume Server Failed to talk with master %s: %v", masterAddress, err)
return "", err
}
case err = <-doneChan:
@@ -265,9 +265,9 @@ func (vs *VolumeServer) doHeartbeat(masterAddress pb.ServerAddress, grpcDialOpti
Volumes: volumeMessages,
HasNoVolumes: len(volumeMessages) == 0,
}
- glog.V(1).Infof("volume server %s:%d stops and deletes all volumes", vs.store.Ip, vs.store.Port)
+ log.V(2).Infof("volume server %s:%d stops and deletes all volumes", vs.store.Ip, vs.store.Port)
if err = stream.Send(emptyBeat); err != nil {
- glog.V(0).Infof("Volume Server Failed to update to master %s: %v", masterAddress, err)
+ log.V(3).Infof("Volume Server Failed to update to master %s: %v", masterAddress, err)
return "", err
}
return
diff --git a/weed/server/volume_grpc_copy.go b/weed/server/volume_grpc_copy.go
index d34a74f55..62d65dbfa 100644
--- a/weed/server/volume_grpc_copy.go
+++ b/weed/server/volume_grpc_copy.go
@@ -11,7 +11,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
"github.com/seaweedfs/seaweedfs/weed/storage/backend"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb"
@@ -30,14 +30,14 @@ func (vs *VolumeServer) VolumeCopy(req *volume_server_pb.VolumeCopyRequest, stre
v := vs.store.GetVolume(needle.VolumeId(req.VolumeId))
if v != nil {
- glog.V(0).Infof("volume %d already exists. deleted before copying...", req.VolumeId)
+ log.V(3).Infof("volume %d already exists. deleted before copying...", req.VolumeId)
err := vs.store.DeleteVolume(needle.VolumeId(req.VolumeId), false)
if err != nil {
return fmt.Errorf("failed to delete existing volume %d: %v", req.VolumeId, err)
}
- glog.V(0).Infof("deleted existing volume %d before copying.", req.VolumeId)
+ log.V(3).Infof("deleted existing volume %d before copying.", req.VolumeId)
}
// the master will not start compaction for read-only volumes, so it is safe to just copy files directly
@@ -96,7 +96,7 @@ func (vs *VolumeServer) VolumeCopy(req *volume_server_pb.VolumeCopyRequest, stre
}
return nil
}); grpcErr != nil {
- glog.V(0).Infof("connect to %s: %v", vs.GetMaster(context.Background()), grpcErr)
+ log.V(3).Infof("connect to %s: %v", vs.GetMaster(context.Background()), grpcErr)
}
if preallocateSize > 0 && !hasRemoteDatFile {
@@ -192,7 +192,7 @@ func (vs *VolumeServer) VolumeCopy(req *volume_server_pb.VolumeCopyRequest, stre
if err = stream.Send(&volume_server_pb.VolumeCopyResponse{
LastAppendAtNs: volFileInfoResp.DatFileTimestampSeconds * uint64(time.Second),
}); err != nil {
- glog.Errorf("send response: %v", err)
+ log.Errorf("send response: %v", err)
}
return err
@@ -257,7 +257,7 @@ func checkCopyFiles(originFileInf *volume_server_pb.ReadVolumeFileStatusResponse
}
func writeToFile(client volume_server_pb.VolumeServer_CopyFileClient, fileName string, wt *util.WriteThrottler, isAppend bool, progressFn storage.ProgressFunc) (modifiedTsNs int64, err error) {
- glog.V(4).Infof("writing to %s", fileName)
+ log.V(-1).Infof("writing to %s", fileName)
flags := os.O_WRONLY | os.O_CREATE | os.O_TRUNC
if isAppend {
flags = os.O_WRONLY | os.O_CREATE
diff --git a/weed/server/volume_grpc_erasure_coding.go b/weed/server/volume_grpc_erasure_coding.go
index 642e8cce3..538fe757d 100644
--- a/weed/server/volume_grpc_erasure_coding.go
+++ b/weed/server/volume_grpc_erasure_coding.go
@@ -10,7 +10,7 @@ import (
"strings"
"time"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb"
@@ -38,7 +38,7 @@ Steps to apply erasure coding to .dat .idx files
// VolumeEcShardsGenerate generates the .ecx and .ec00 ~ .ec13 files
func (vs *VolumeServer) VolumeEcShardsGenerate(ctx context.Context, req *volume_server_pb.VolumeEcShardsGenerateRequest) (*volume_server_pb.VolumeEcShardsGenerateResponse, error) {
- glog.V(0).Infof("VolumeEcShardsGenerate: %v", req)
+ log.V(3).Infof("VolumeEcShardsGenerate: %v", req)
v := vs.store.GetVolume(needle.VolumeId(req.VolumeId))
if v == nil {
@@ -96,7 +96,7 @@ func (vs *VolumeServer) VolumeEcShardsGenerate(ctx context.Context, req *volume_
// VolumeEcShardsRebuild generates the any of the missing .ec00 ~ .ec13 files
func (vs *VolumeServer) VolumeEcShardsRebuild(ctx context.Context, req *volume_server_pb.VolumeEcShardsRebuildRequest) (*volume_server_pb.VolumeEcShardsRebuildResponse, error) {
- glog.V(0).Infof("VolumeEcShardsRebuild: %v", req)
+ log.V(3).Infof("VolumeEcShardsRebuild: %v", req)
baseFileName := erasure_coding.EcShardBaseFileName(req.Collection, int(req.VolumeId))
@@ -138,7 +138,7 @@ func (vs *VolumeServer) VolumeEcShardsRebuild(ctx context.Context, req *volume_s
// VolumeEcShardsCopy copy the .ecx and some ec data slices
func (vs *VolumeServer) VolumeEcShardsCopy(ctx context.Context, req *volume_server_pb.VolumeEcShardsCopyRequest) (*volume_server_pb.VolumeEcShardsCopyResponse, error) {
- glog.V(0).Infof("VolumeEcShardsCopy: %v", req)
+ log.V(3).Infof("VolumeEcShardsCopy: %v", req)
var location *storage.DiskLocation
if req.CopyEcxFile {
@@ -205,11 +205,11 @@ func (vs *VolumeServer) VolumeEcShardsDelete(ctx context.Context, req *volume_se
bName := erasure_coding.EcShardBaseFileName(req.Collection, int(req.VolumeId))
- glog.V(0).Infof("ec volume %s shard delete %v", bName, req.ShardIds)
+ log.V(3).Infof("ec volume %s shard delete %v", bName, req.ShardIds)
for _, location := range vs.store.Locations {
if err := deleteEcShardIdsForEachLocation(bName, location, req.ShardIds); err != nil {
- glog.Errorf("deleteEcShards from %s %s.%v: %v", location.Directory, bName, req.ShardIds, err)
+ log.Errorf("deleteEcShards from %s %s.%v: %v", location.Directory, bName, req.ShardIds, err)
return nil, err
}
}
@@ -289,15 +289,15 @@ func checkEcVolumeStatus(bName string, location *storage.DiskLocation) (hasEcxFi
func (vs *VolumeServer) VolumeEcShardsMount(ctx context.Context, req *volume_server_pb.VolumeEcShardsMountRequest) (*volume_server_pb.VolumeEcShardsMountResponse, error) {
- glog.V(0).Infof("VolumeEcShardsMount: %v", req)
+ log.V(3).Infof("VolumeEcShardsMount: %v", req)
for _, shardId := range req.ShardIds {
err := vs.store.MountEcShards(req.Collection, needle.VolumeId(req.VolumeId), erasure_coding.ShardId(shardId))
if err != nil {
- glog.Errorf("ec shard mount %v: %v", req, err)
+ log.Errorf("ec shard mount %v: %v", req, err)
} else {
- glog.V(2).Infof("ec shard mount %v", req)
+ log.V(1).Infof("ec shard mount %v", req)
}
if err != nil {
@@ -310,15 +310,15 @@ func (vs *VolumeServer) VolumeEcShardsMount(ctx context.Context, req *volume_ser
func (vs *VolumeServer) VolumeEcShardsUnmount(ctx context.Context, req *volume_server_pb.VolumeEcShardsUnmountRequest) (*volume_server_pb.VolumeEcShardsUnmountResponse, error) {
- glog.V(0).Infof("VolumeEcShardsUnmount: %v", req)
+ log.V(3).Infof("VolumeEcShardsUnmount: %v", req)
for _, shardId := range req.ShardIds {
err := vs.store.UnmountEcShards(needle.VolumeId(req.VolumeId), erasure_coding.ShardId(shardId))
if err != nil {
- glog.Errorf("ec shard unmount %v: %v", req, err)
+ log.Errorf("ec shard unmount %v: %v", req, err)
} else {
- glog.V(2).Infof("ec shard unmount %v", req)
+ log.V(1).Infof("ec shard unmount %v", req)
}
if err != nil {
@@ -399,7 +399,7 @@ func (vs *VolumeServer) VolumeEcShardRead(req *volume_server_pb.VolumeEcShardRea
func (vs *VolumeServer) VolumeEcBlobDelete(ctx context.Context, req *volume_server_pb.VolumeEcBlobDeleteRequest) (*volume_server_pb.VolumeEcBlobDeleteResponse, error) {
- glog.V(0).Infof("VolumeEcBlobDelete: %v", req)
+ log.V(3).Infof("VolumeEcBlobDelete: %v", req)
resp := &volume_server_pb.VolumeEcBlobDeleteResponse{}
@@ -429,7 +429,7 @@ func (vs *VolumeServer) VolumeEcBlobDelete(ctx context.Context, req *volume_serv
// VolumeEcShardsToVolume generates the .idx, .dat files from .ecx, .ecj and .ec01 ~ .ec14 files
func (vs *VolumeServer) VolumeEcShardsToVolume(ctx context.Context, req *volume_server_pb.VolumeEcShardsToVolumeRequest) (*volume_server_pb.VolumeEcShardsToVolumeResponse, error) {
- glog.V(0).Infof("VolumeEcShardsToVolume: %v", req)
+ log.V(3).Infof("VolumeEcShardsToVolume: %v", req)
// collect .ec00 ~ .ec09 files
shardFileNames := make([]string, erasure_coding.DataShardsCount)
diff --git a/weed/server/volume_grpc_query.go b/weed/server/volume_grpc_query.go
index a1abcb8eb..eb2cd83a0 100644
--- a/weed/server/volume_grpc_query.go
+++ b/weed/server/volume_grpc_query.go
@@ -1,7 +1,7 @@
package weed_server
import (
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb"
"github.com/seaweedfs/seaweedfs/weed/query/json"
@@ -15,7 +15,7 @@ func (vs *VolumeServer) Query(req *volume_server_pb.QueryRequest, stream volume_
vid, id_cookie, err := operation.ParseFileId(fid)
if err != nil {
- glog.V(0).Infof("volume query failed to parse fid %s: %v", fid, err)
+ log.V(3).Infof("volume query failed to parse fid %s: %v", fid, err)
return err
}
@@ -25,12 +25,12 @@ func (vs *VolumeServer) Query(req *volume_server_pb.QueryRequest, stream volume_
cookie := n.Cookie
if _, err := vs.store.ReadVolumeNeedle(volumeId, n, nil, nil); err != nil {
- glog.V(0).Infof("volume query failed to read fid %s: %v", fid, err)
+ log.V(3).Infof("volume query failed to read fid %s: %v", fid, err)
return err
}
if n.Cookie != cookie {
- glog.V(0).Infof("volume query failed to read fid cookie %s: %v", fid, err)
+ log.V(3).Infof("volume query failed to read fid cookie %s: %v", fid, err)
return err
}
diff --git a/weed/server/volume_grpc_tail.go b/weed/server/volume_grpc_tail.go
index b44d7d248..b2c11820d 100644
--- a/weed/server/volume_grpc_tail.go
+++ b/weed/server/volume_grpc_tail.go
@@ -6,7 +6,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/pb"
"time"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb"
"github.com/seaweedfs/seaweedfs/weed/storage"
@@ -21,7 +21,7 @@ func (vs *VolumeServer) VolumeTailSender(req *volume_server_pb.VolumeTailSenderR
return fmt.Errorf("not found volume id %d", req.VolumeId)
}
- defer glog.V(1).Infof("tailing volume %d finished", v.Id)
+ defer log.V(2).Infof("tailing volume %d finished", v.Id)
lastTimestampNs := req.SinceNs
drainingSeconds := req.IdleTimeoutSeconds
@@ -29,7 +29,7 @@ func (vs *VolumeServer) VolumeTailSender(req *volume_server_pb.VolumeTailSenderR
for {
lastProcessedTimestampNs, err := sendNeedlesSince(stream, v, lastTimestampNs)
if err != nil {
- glog.Infof("sendNeedlesSince: %v", err)
+ log.Infof("sendNeedlesSince: %v", err)
return fmt.Errorf("streamFollow: %v", err)
}
time.Sleep(2 * time.Second)
@@ -43,11 +43,11 @@ func (vs *VolumeServer) VolumeTailSender(req *volume_server_pb.VolumeTailSenderR
if drainingSeconds <= 0 {
return nil
}
- glog.V(1).Infof("tailing volume %d drains requests with %d seconds remaining", v.Id, drainingSeconds)
+ log.V(2).Infof("tailing volume %d drains requests with %d seconds remaining", v.Id, drainingSeconds)
} else {
lastTimestampNs = lastProcessedTimestampNs
drainingSeconds = req.IdleTimeoutSeconds
- glog.V(1).Infof("tailing volume %d resets draining wait time to %d seconds", v.Id, drainingSeconds)
+ log.V(2).Infof("tailing volume %d resets draining wait time to %d seconds", v.Id, drainingSeconds)
}
}
@@ -88,7 +88,7 @@ func (vs *VolumeServer) VolumeTailReceiver(ctx context.Context, req *volume_serv
return resp, fmt.Errorf("receiver not found volume id %d", req.VolumeId)
}
- defer glog.V(1).Infof("receive tailing volume %d finished", v.Id)
+ defer log.V(2).Infof("receive tailing volume %d finished", v.Id)
return resp, operation.TailVolumeFromSource(pb.ServerAddress(req.SourceVolumeServer), vs.grpcDialOption, v.Id, req.SinceNs, int(req.IdleTimeoutSeconds), func(n *needle.Needle) error {
_, err := vs.store.WriteVolumeNeedle(v.Id, n, false, false)
diff --git a/weed/server/volume_grpc_vacuum.go b/weed/server/volume_grpc_vacuum.go
index 990611052..ad3f92671 100644
--- a/weed/server/volume_grpc_vacuum.go
+++ b/weed/server/volume_grpc_vacuum.go
@@ -10,7 +10,7 @@ import (
"runtime"
"github.com/prometheus/procfs"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb"
"github.com/seaweedfs/seaweedfs/weed/storage/needle"
)
@@ -26,7 +26,7 @@ func (vs *VolumeServer) VacuumVolumeCheck(ctx context.Context, req *volume_serve
resp.GarbageRatio = garbageRatio
if err != nil {
- glog.V(3).Infof("check volume %d: %v", req.VolumeId, err)
+ log.V(0).Infof("check volume %d: %v", req.VolumeId, err)
}
return resp, err
@@ -62,15 +62,15 @@ func (vs *VolumeServer) VacuumVolumeCompact(req *volume_server_pb.VacuumVolumeCo
stats.VolumeServerVacuumingCompactCounter.WithLabelValues(strconv.FormatBool(err == nil && sendErr == nil)).Inc()
if err != nil {
- glog.Errorf("failed compact volume %d: %v", req.VolumeId, err)
+ log.Errorf("failed compact volume %d: %v", req.VolumeId, err)
return err
}
if sendErr != nil {
- glog.Errorf("failed compact volume %d report progress: %v", req.VolumeId, sendErr)
+ log.Errorf("failed compact volume %d report progress: %v", req.VolumeId, sendErr)
return sendErr
}
- glog.V(1).Infof("compact volume %d", req.VolumeId)
+ log.V(2).Infof("compact volume %d", req.VolumeId)
return nil
}
@@ -86,9 +86,9 @@ func (vs *VolumeServer) VacuumVolumeCommit(ctx context.Context, req *volume_serv
readOnly, volumeSize, err := vs.store.CommitCompactVolume(needle.VolumeId(req.VolumeId))
if err != nil {
- glog.Errorf("failed commit volume %d: %v", req.VolumeId, err)
+ log.Errorf("failed commit volume %d: %v", req.VolumeId, err)
} else {
- glog.V(1).Infof("commit volume %d", req.VolumeId)
+ log.V(2).Infof("commit volume %d", req.VolumeId)
}
stats.VolumeServerVacuumingCommitCounter.WithLabelValues(strconv.FormatBool(err == nil)).Inc()
resp.IsReadOnly = readOnly
@@ -104,9 +104,9 @@ func (vs *VolumeServer) VacuumVolumeCleanup(ctx context.Context, req *volume_ser
err := vs.store.CommitCleanupVolume(needle.VolumeId(req.VolumeId))
if err != nil {
- glog.Errorf("failed cleanup volume %d: %v", req.VolumeId, err)
+ log.Errorf("failed cleanup volume %d: %v", req.VolumeId, err)
} else {
- glog.V(1).Infof("cleanup volume %d", req.VolumeId)
+ log.V(2).Infof("cleanup volume %d", req.VolumeId)
}
return resp, err
diff --git a/weed/server/volume_server.go b/weed/server/volume_server.go
index a3f072eb7..04bbd90a0 100644
--- a/weed/server/volume_server.go
+++ b/weed/server/volume_server.go
@@ -14,7 +14,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/stats"
"github.com/seaweedfs/seaweedfs/weed/util"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/security"
"github.com/seaweedfs/seaweedfs/weed/storage"
)
@@ -140,23 +140,23 @@ func NewVolumeServer(adminMux, publicMux *http.ServeMux, ip string,
}
func (vs *VolumeServer) SetStopping() {
- glog.V(0).Infoln("Stopping volume server...")
+ log.V(3).Infoln("Stopping volume server...")
vs.store.SetStopping()
}
func (vs *VolumeServer) LoadNewVolumes() {
- glog.V(0).Infoln(" Loading new volume ids ...")
+ log.V(3).Infoln(" Loading new volume ids ...")
vs.store.LoadNewVolumes()
}
func (vs *VolumeServer) Shutdown() {
- glog.V(0).Infoln("Shutting down volume server...")
+ log.V(3).Infoln("Shutting down volume server...")
vs.store.Close()
- glog.V(0).Infoln("Shut down successfully!")
+ log.V(3).Infoln("Shut down successfully!")
}
func (vs *VolumeServer) Reload() {
- glog.V(0).Infoln("Reload volume server...")
+ log.V(3).Infoln("Reload volume server...")
util.LoadConfiguration("security", false)
v := util.GetViper()
diff --git a/weed/server/volume_server_handlers.go b/weed/server/volume_server_handlers.go
index 22ef0e1c8..df08cde88 100644
--- a/weed/server/volume_server_handlers.go
+++ b/weed/server/volume_server_handlers.go
@@ -10,7 +10,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/util"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/security"
"github.com/seaweedfs/seaweedfs/weed/stats"
)
@@ -56,12 +56,12 @@ func (vs *VolumeServer) privateStoreHandler(w http.ResponseWriter, r *http.Reque
for vs.concurrentDownloadLimit != 0 && inFlightDownloadSize > vs.concurrentDownloadLimit {
select {
case <-r.Context().Done():
- glog.V(4).Infof("request cancelled from %s: %v", r.RemoteAddr, r.Context().Err())
+ log.V(-1).Infof("request cancelled from %s: %v", r.RemoteAddr, r.Context().Err())
w.WriteHeader(util.HttpStatusCancelled)
vs.inFlightDownloadDataLimitCond.L.Unlock()
return
default:
- glog.V(4).Infof("wait because inflight download data %d > %d", inFlightDownloadSize, vs.concurrentDownloadLimit)
+ log.V(-1).Infof("wait because inflight download data %d > %d", inFlightDownloadSize, vs.concurrentDownloadLimit)
vs.inFlightDownloadDataLimitCond.Wait()
}
inFlightDownloadSize = atomic.LoadInt64(&vs.inFlightDownloadDataSize)
@@ -83,11 +83,11 @@ func (vs *VolumeServer) privateStoreHandler(w http.ResponseWriter, r *http.Reque
if startTime.Add(vs.inflightUploadDataTimeout).Before(time.Now()) {
vs.inFlightUploadDataLimitCond.L.Unlock()
err := fmt.Errorf("reject because inflight upload data %d > %d, and wait timeout", inFlightUploadDataSize, vs.concurrentUploadLimit)
- glog.V(1).Infof("too many requests: %v", err)
+ log.V(2).Infof("too many requests: %v", err)
writeJsonError(w, r, http.StatusTooManyRequests, err)
return
}
- glog.V(4).Infof("wait because inflight upload data %d > %d", inFlightUploadDataSize, vs.concurrentUploadLimit)
+ log.V(-1).Infof("wait because inflight upload data %d > %d", inFlightUploadDataSize, vs.concurrentUploadLimit)
vs.inFlightUploadDataLimitCond.Wait()
inFlightUploadDataSize = atomic.LoadInt64(&vs.inFlightUploadDataSize)
}
@@ -149,7 +149,7 @@ func (vs *VolumeServer) publicReadOnlyHandler(w http.ResponseWriter, r *http.Req
vs.inFlightDownloadDataLimitCond.L.Lock()
inFlightDownloadSize := atomic.LoadInt64(&vs.inFlightDownloadDataSize)
for vs.concurrentDownloadLimit != 0 && inFlightDownloadSize > vs.concurrentDownloadLimit {
- glog.V(4).Infof("wait because inflight download data %d > %d", inFlightDownloadSize, vs.concurrentDownloadLimit)
+ log.V(-1).Infof("wait because inflight download data %d > %d", inFlightDownloadSize, vs.concurrentDownloadLimit)
vs.inFlightDownloadDataLimitCond.Wait()
inFlightDownloadSize = atomic.LoadInt64(&vs.inFlightDownloadDataSize)
}
@@ -182,17 +182,17 @@ func (vs *VolumeServer) maybeCheckJwtAuthorization(r *http.Request, vid, fid str
tokenStr := security.GetJwt(r)
if tokenStr == "" {
- glog.V(1).Infof("missing jwt from %s", r.RemoteAddr)
+ log.V(2).Infof("missing jwt from %s", r.RemoteAddr)
return false
}
token, err := security.DecodeJwt(signingKey, tokenStr, &security.SeaweedFileIdClaims{})
if err != nil {
- glog.V(1).Infof("jwt verification error from %s: %v", r.RemoteAddr, err)
+ log.V(2).Infof("jwt verification error from %s: %v", r.RemoteAddr, err)
return false
}
if !token.Valid {
- glog.V(1).Infof("jwt invalid from %s: %v", r.RemoteAddr, tokenStr)
+ log.V(2).Infof("jwt invalid from %s: %v", r.RemoteAddr, tokenStr)
return false
}
@@ -202,6 +202,6 @@ func (vs *VolumeServer) maybeCheckJwtAuthorization(r *http.Request, vid, fid str
}
return sc.Fid == vid+","+fid
}
- glog.V(1).Infof("unexpected jwt from %s: %v", r.RemoteAddr, tokenStr)
+ log.V(2).Infof("unexpected jwt from %s: %v", r.RemoteAddr, tokenStr)
return false
}
diff --git a/weed/server/volume_server_handlers_read.go b/weed/server/volume_server_handlers_read.go
index 15d639f49..bee9f757e 100644
--- a/weed/server/volume_server_handlers_read.go
+++ b/weed/server/volume_server_handlers_read.go
@@ -20,7 +20,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/storage/types"
"github.com/seaweedfs/seaweedfs/weed/util/mem"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/images"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/stats"
@@ -53,30 +53,30 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
volumeId, err := needle.NewVolumeId(vid)
if err != nil {
- glog.V(2).Infof("parsing vid %s: %v", r.URL.Path, err)
+ log.V(1).Infof("parsing vid %s: %v", r.URL.Path, err)
w.WriteHeader(http.StatusBadRequest)
return
}
err = n.ParsePath(fid)
if err != nil {
- glog.V(2).Infof("parsing fid %s: %v", r.URL.Path, err)
+ log.V(1).Infof("parsing fid %s: %v", r.URL.Path, err)
w.WriteHeader(http.StatusBadRequest)
return
}
- // glog.V(4).Infoln("volume", volumeId, "reading", n)
+ // log.V(-1).Infoln("volume", volumeId, "reading", n)
hasVolume := vs.store.HasVolume(volumeId)
_, hasEcVolume := vs.store.FindEcVolume(volumeId)
if !hasVolume && !hasEcVolume {
if vs.ReadMode == "local" {
- glog.V(0).Infoln("volume is not local:", err, r.URL.Path)
+ log.V(3).Infoln("volume is not local:", err, r.URL.Path)
NotFound(w)
return
}
lookupResult, err := operation.LookupVolumeId(vs.GetMaster, vs.grpcDialOption, volumeId.String())
- glog.V(2).Infoln("volume", volumeId, "found on", lookupResult, "error", err)
+ log.V(1).Infoln("volume", volumeId, "found on", lookupResult, "error", err)
if err != nil || len(lookupResult.Locations) <= 0 {
- glog.V(0).Infoln("lookup error:", err, r.URL.Path)
+ log.V(3).Infoln("lookup error:", err, r.URL.Path)
NotFound(w)
return
}
@@ -88,7 +88,7 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
r.URL.Scheme = u.Scheme
request, err := http.NewRequest(http.MethodGet, r.URL.String(), nil)
if err != nil {
- glog.V(0).Infof("failed to instance http request of url %s: %v", r.URL.String(), err)
+ log.V(3).Infof("failed to instance http request of url %s: %v", r.URL.String(), err)
InternalError(w)
return
}
@@ -100,7 +100,7 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
response, err := util_http.GetGlobalHttpClient().Do(request)
if err != nil {
- glog.V(0).Infof("request remote url %s: %v", r.URL.String(), err)
+ log.V(3).Infof("request remote url %s: %v", r.URL.String(), err)
InternalError(w)
return
}
@@ -156,12 +156,12 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
}()
if err != nil && err != storage.ErrorDeleted && hasVolume {
- glog.V(4).Infof("read needle: %v", err)
+ log.V(-1).Infof("read needle: %v", err)
// start to fix it from other replicas, if not deleted and hasVolume and is not a replicated request
}
- // glog.V(4).Infoln("read bytes", count, "error", err)
+ // log.V(-1).Infoln("read bytes", count, "error", err)
if err != nil || count < 0 {
- glog.V(3).Infof("read %s isNormalVolume %v error: %v", r.URL.Path, hasVolume, err)
+ log.V(0).Infof("read %s isNormalVolume %v error: %v", r.URL.Path, hasVolume, err)
if err == storage.ErrorNotFound || err == storage.ErrorDeleted {
NotFound(w)
} else {
@@ -170,7 +170,7 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
return
}
if n.Cookie != cookie {
- glog.V(0).Infof("request %s with cookie:%x expected:%x from %s agent %s", r.URL.Path, cookie, n.Cookie, r.RemoteAddr, r.UserAgent())
+ log.V(3).Infof("request %s with cookie:%x expected:%x from %s agent %s", r.URL.Path, cookie, n.Cookie, r.RemoteAddr, r.UserAgent())
NotFound(w)
return
}
@@ -195,7 +195,7 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
pairMap := make(map[string]string)
err = json.Unmarshal(n.Pairs, &pairMap)
if err != nil {
- glog.V(0).Infoln("Unmarshal pairs error:", err)
+ log.V(3).Infoln("Unmarshal pairs error:", err)
}
for k, v := range pairMap {
w.Header().Set(k, v)
@@ -225,7 +225,7 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
_, _, _, _, shouldCrop := shouldCropImages(ext, r)
if shouldResize || shouldCrop {
if n.Data, err = util.DecompressData(n.Data); err != nil {
- glog.V(0).Infoln("ungzip error:", err, r.URL.Path)
+ log.V(3).Infoln("ungzip error:", err, r.URL.Path)
}
// } else if strings.Contains(r.Header.Get("Accept-Encoding"), "zstd") && util.IsZstdContent(n.Data) {
// w.Header().Set("Content-Encoding", "zstd")
@@ -233,7 +233,7 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
w.Header().Set("Content-Encoding", "gzip")
} else {
if n.Data, err = util.DecompressData(n.Data); err != nil {
- glog.V(0).Infoln("uncompress error:", err, r.URL.Path)
+ log.V(3).Infoln("uncompress error:", err, r.URL.Path)
}
}
}
@@ -242,7 +242,7 @@ func (vs *VolumeServer) GetOrHeadHandler(w http.ResponseWriter, r *http.Request)
rs := conditionallyCropImages(bytes.NewReader(n.Data), ext, r)
rs = conditionallyResizeImages(rs, ext, r)
if e := writeResponseContent(filename, mtype, rs, w, r); e != nil {
- glog.V(2).Infoln("response write error:", e)
+ log.V(1).Infoln("response write error:", e)
}
} else {
vs.streamWriteResponseContent(filename, mtype, volumeId, n, w, r, readOption)
@@ -274,7 +274,7 @@ func (vs *VolumeServer) tryHandleChunkedFile(n *needle.Needle, fileName string,
chunkManifest, e := operation.LoadChunkManifest(n.Data, n.IsCompressed())
if e != nil {
- glog.V(0).Infof("load chunked manifest (%s) error: %v", r.URL.Path, e)
+ log.V(3).Infof("load chunked manifest (%s) error: %v", r.URL.Path, e)
return false
}
if fileName == "" && chunkManifest.Name != "" {
@@ -302,7 +302,7 @@ func (vs *VolumeServer) tryHandleChunkedFile(n *needle.Needle, fileName string,
rs = conditionallyResizeImages(rs, ext, r)
if e := writeResponseContent(fileName, mType, rs, w, r); e != nil {
- glog.V(2).Infoln("response write error:", e)
+ log.V(1).Infoln("response write error:", e)
}
return true
}
@@ -343,7 +343,7 @@ func conditionallyCropImages(originalDataReaderSeeker io.ReadSeeker, ext string,
var err error
rs, err = images.Cropped(ext, rs, x1, y1, x2, y2)
if err != nil {
- glog.Errorf("Cropping images error: %s", err)
+ log.Errorf("Cropping images error: %s", err)
}
}
return rs
diff --git a/weed/server/volume_server_handlers_write.go b/weed/server/volume_server_handlers_write.go
index 7f0fcc871..46db10119 100644
--- a/weed/server/volume_server_handlers_write.go
+++ b/weed/server/volume_server_handlers_write.go
@@ -8,7 +8,7 @@ import (
"strings"
"time"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/storage/needle"
"github.com/seaweedfs/seaweedfs/weed/topology"
@@ -17,7 +17,7 @@ import (
func (vs *VolumeServer) PostHandler(w http.ResponseWriter, r *http.Request) {
if e := r.ParseForm(); e != nil {
- glog.V(0).Infoln("form parse error:", e)
+ log.V(3).Infoln("form parse error:", e)
writeJsonError(w, r, http.StatusBadRequest, e)
return
}
@@ -25,7 +25,7 @@ func (vs *VolumeServer) PostHandler(w http.ResponseWriter, r *http.Request) {
vid, fid, _, _, _ := parseURLPath(r.URL.Path)
volumeId, ve := needle.NewVolumeId(vid)
if ve != nil {
- glog.V(0).Infoln("NewVolumeId error:", ve)
+ log.V(3).Infoln("NewVolumeId error:", ve)
writeJsonError(w, r, http.StatusBadRequest, ve)
return
}
@@ -81,7 +81,7 @@ func (vs *VolumeServer) DeleteHandler(w http.ResponseWriter, r *http.Request) {
return
}
- // glog.V(2).Infof("volume %s deleting %s", vid, n)
+ // log.V(1).Infof("volume %s deleting %s", vid, n)
cookie := n.Cookie
@@ -102,7 +102,7 @@ func (vs *VolumeServer) DeleteHandler(w http.ResponseWriter, r *http.Request) {
}
if n.Cookie != cookie {
- glog.V(0).Infoln("delete", r.URL.Path, "with unmaching cookie from ", r.RemoteAddr, "agent", r.UserAgent())
+ log.V(3).Infoln("delete", r.URL.Path, "with unmaching cookie from ", r.RemoteAddr, "agent", r.UserAgent())
writeJsonError(w, r, http.StatusBadRequest, errors.New("File Random Cookie does not match."))
return
}
diff --git a/weed/server/webdav_server.go b/weed/server/webdav_server.go
index dbe6dfed5..fece54f69 100644
--- a/weed/server/webdav_server.go
+++ b/weed/server/webdav_server.go
@@ -20,7 +20,7 @@ import (
"github.com/seaweedfs/seaweedfs/weed/util/chunk_cache"
"github.com/seaweedfs/seaweedfs/weed/filer"
- "github.com/seaweedfs/seaweedfs/weed/glog"
+ "github.com/seaweedfs/seaweedfs/weed/util/log"
"github.com/seaweedfs/seaweedfs/weed/security"
)
@@ -174,7 +174,7 @@ func clearName(name string) (string, error) {
func (fs *WebDavFileSystem) Mkdir(ctx context.Context, fullDirPath string, perm os.FileMode) error {
- glog.V(2).Infof("WebDavFileSystem.Mkdir %v", fullDirPath)
+ log.V(1).Infof("WebDavFileSystem.Mkdir %v", fullDirPath)
if !strings.HasSuffix(fullDirPath, "/") {
fullDirPath += "/"
@@ -208,7 +208,7 @@ func (fs *WebDavFileSystem) Mkdir(ctx context.Context, fullDirPath string, perm
Signatures: []int32{fs.signature},
}
- glog.V(1).Infof("mkdir: %v", request)
+ log.V(2).Infof("mkdir: %v", request)
if err := filer_pb.CreateEntry(client, request); err != nil {
return fmt.Errorf("mkdir %s/%s: %v", dir, name, err)
}
@@ -218,7 +218,7 @@ func (fs *WebDavFileSystem) Mkdir(ctx context.Context, fullDirPath string, perm
}
func (fs *WebDavFileSystem) OpenFile(ctx context.Context, fullFilePath string, flag int, perm os.FileMode) (webdav.File, error) {
- glog.V(2).Infof("WebDavFileSystem.OpenFile %v %x", fullFilePath, flag)
+ log.V(1).Infof("WebDavFileSystem.OpenFile %v %x", fullFilePath, flag)
var err error
if fullFilePath, err = clearName(fullFilePath); err != nil {
@@ -305,14 +305,14 @@ func (fs *WebDavFileSystem) removeAll(ctx context.Context, fullFilePath string)
func (fs *WebDavFileSystem) RemoveAll(ctx context.Context, name string) error {
- glog.V(2).Infof("WebDavFileSystem.RemoveAll %v", name)
+ log.V(1).Infof("WebDavFileSystem.RemoveAll %v", name)
return fs.removeAll(ctx, name)
}
func (fs *WebDavFileSystem) Rename(ctx context.Context, oldName, newName string) error {
- glog.V(2).Infof("WebDavFileSystem.Rename %v to %v", oldName, newName)
+ log.V(1).Infof("WebDavFileSystem.Rename %v to %v", oldName, newName)
var err error
if oldName, err = clearName(oldName); err != nil {
@@ -397,7 +397,7 @@ func (fs *WebDavFileSystem) stat(ctx context.Context, fullFilePath string) (os.F
}
func (fs *WebDavFileSystem) Stat(ctx context.Context, name string) (os.FileInfo, error) {
- glog.V(2).Infof("WebDavFileSystem.Stat %v", name)
+ log.V(1).Infof("WebDavFileSystem.Stat %v", name)
return fs.stat(ctx, name)
}
@@ -405,7 +405,7 @@ func (fs *WebDavFileSystem) Stat(ctx context.Context, name string) (os.FileInfo,
func (f *WebDavFile) saveDataAsChunk(reader io.Reader, name string, offset int64, tsNs int64) (chunk *filer_pb.FileChunk, err error) {
uploader, uploaderErr := operation.NewUploader()
if uploaderErr != nil {
- glog.V(0).Infof("upload data %v: %v", f.name, uploaderErr)
+ log.V(3).Infof("upload data %v: %v", f.name, uploaderErr)
return nil, fmt.Errorf("upload data: %v", uploaderErr)
}
@@ -432,11 +432,11 @@ func (f *WebDavFile) saveDataAsChunk(reader io.Reader, name string, offset int64
)
if flushErr != nil {
- glog.V(0).Infof("upload data %v: %v", f.name, flushErr)
+ log.V(3).Infof("upload data %v: %v", f.name, flushErr)
return nil, fmt.Errorf("upload data: %v", flushErr)
}
if uploadResult.Error != "" {
- glog.V(0).Infof("upload failure %v: %v", f.name, flushErr)
+ log.V(3).Infof("upload failure %v: %v", f.name, flushErr)
return nil, fmt.Errorf("upload result: %v", uploadResult.Error)
}
return uploadResult.ToPbFileChunk(fileId, offset, tsNs), nil
@@ -444,7 +444,7 @@ func (f *WebDavFile) saveDataAsChunk(reader io.Reader, name string, offset int64
func (f *WebDavFile) Write(buf []byte) (int, error) {
- glog.V(2).Infof("WebDavFileSystem.Write %v", f.name)
+ log.V(1).Infof("WebDavFileSystem.Write %v", f.name)
fullPath := util.FullPath(f.name)
dir, _ := fullPath.DirAndName()
@@ -471,7 +471,7 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
if flushErr != nil {
if f.entry.Attributes.Mtime == 0 {
if err := f.fs.removeAll(ctx, f.name); err != nil {
- glog.Errorf("bufWriter.Flush remove file error: %+v", f.name)
+ log.Errorf("bufWriter.Flush remove file error: %+v", f.name)
}
}
return fmt.Errorf("%s upload result: %v", f.name, flushErr)
@@ -487,7 +487,7 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
manifestedChunks, manifestErr := filer.MaybeManifestize(f.saveDataAsChunk, f.entry.GetChunks())
if manifestErr != nil {
// not good, but should be ok
- glog.V(0).Infof("file %s close MaybeManifestize: %v", f.name, manifestErr)
+ log.V(3).Infof("file %s close MaybeManifestize: %v", f.name, manifestErr)
} else {
f.entry.Chunks = manifestedChunks
}
@@ -515,7 +515,7 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
if err == nil {
f.entry.Attributes.FileSize = uint64(max(f.off+int64(written), int64(f.entry.Attributes.FileSize)))
- glog.V(3).Infof("WebDavFileSystem.Write %v: written [%d,%d)", f.name, f.off, f.off+int64(len(buf)))
+ log.V(0).Infof("WebDavFileSystem.Write %v: written [%d,%d)", f.name, f.off, f.off+int64(len(buf)))
f.off += int64(written)
}
@@ -524,7 +524,7 @@ func (f *WebDavFile) Write(buf []byte) (int, error) {
func (f *WebDavFile) Close() error {
- glog.V(2).Infof("WebDavFileSystem.Close %v", f.name)
+ log.V(1).Infof("WebDavFileSystem.Close %v", f.name)
if f.bufWriter == nil {
return nil
}
@@ -540,7 +540,7 @@ func (f *WebDavFile) Close() error {
func (f *WebDavFile) Read(p []byte) (readSize int, err error) {
- glog.V(2).Infof("WebDavFileSystem.Read %v", f.name)
+ log.V(1).Infof("WebDavFileSystem.Read %v", f.name)
if f.entry == nil {
f.entry, err = filer_pb.GetEntry(f.fs, util.FullPath(f.name))
@@ -566,11 +566,11 @@ func (f *WebDavFile) Read(p []byte) (readSize int, err error) {
readSize, err = f.reader.ReadAt(p, f.off)
- glog.V(3).Infof("WebDavFileSystem.Read %v: [%d,%d)", f.name, f.off, f.off+int64(readSize))
+ log.V(0).Infof("WebDavFileSystem.Read %v: [%d,%d)", f.name, f.off, f.off+int64(readSize))
f.off += int64(readSize)
if err != nil && err != io.EOF {
- glog.Errorf("file read %s: %v", f.name, err)
+ log.Errorf("file read %s: %v", f.name, err)
}
return
@@ -579,7 +579,7 @@ 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)
+ log.V(1).Infof("WebDavFileSystem.Readdir %v count %d", f.name, count)
dir, _ := util.FullPath(f.name).DirAndName()
@@ -595,7 +595,7 @@ func (f *WebDavFile) Readdir(count int) (ret []os.FileInfo, err error) {
if !strings.HasSuffix(fi.name, "/") && fi.IsDir() {
fi.name += "/"
}
- glog.V(4).Infof("entry: %v", fi.name)
+ log.V(-1).Infof("entry: %v", fi.name)
ret = append(ret, &fi)
return nil
})
@@ -625,7 +625,7 @@ func (f *WebDavFile) Readdir(count int) (ret []os.FileInfo, err error) {
func (f *WebDavFile) Seek(offset int64, whence int) (int64, error) {
- glog.V(2).Infof("WebDavFile.Seek %v %v %v", f.name, offset, whence)
+ log.V(1).Infof("WebDavFile.Seek %v %v %v", f.name, offset, whence)
ctx := context.Background()
@@ -646,7 +646,7 @@ func (f *WebDavFile) Seek(offset int64, whence int) (int64, error) {
func (f *WebDavFile) Stat() (os.FileInfo, error) {
- glog.V(2).Infof("WebDavFile.Stat %v", f.name)
+ log.V(1).Infof("WebDavFile.Stat %v", f.name)
ctx := context.Background()