diff --git a/weed/server/master_grpc_server_volume.go b/weed/server/master_grpc_server_volume.go index 156afd4a1..3a4951cc5 100644 --- a/weed/server/master_grpc_server_volume.go +++ b/weed/server/master_grpc_server_volume.go @@ -4,15 +4,68 @@ import ( "context" "fmt" "github.com/chrislusf/raft" - "github.com/chrislusf/seaweedfs/weed/storage/types" + "reflect" + "sync" + "time" + "github.com/chrislusf/seaweedfs/weed/glog" "github.com/chrislusf/seaweedfs/weed/pb/master_pb" "github.com/chrislusf/seaweedfs/weed/security" "github.com/chrislusf/seaweedfs/weed/storage/needle" "github.com/chrislusf/seaweedfs/weed/storage/super_block" + "github.com/chrislusf/seaweedfs/weed/storage/types" "github.com/chrislusf/seaweedfs/weed/topology" ) +func (ms *MasterServer) ProcessGrowRequest() { + go func() { + filter := sync.Map{} + for { + req, ok := <-ms.vgCh + if !ok { + break + } + + if !ms.Topo.IsLeader() { + //discard buffered requests + time.Sleep(time.Second * 1) + continue + } + + // filter out identical requests being processed + found := false + filter.Range(func(k, v interface{}) bool { + if reflect.DeepEqual(k, req) { + found = true + } + return !found + }) + + // not atomic but it's okay + if !found && ms.shouldVolumeGrow(req.Option) { + filter.Store(req, nil) + // we have lock called inside vg + go func() { + glog.V(1).Infoln("starting automatic volume grow") + start := time.Now() + _, 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)) + + if req.ErrCh != nil { + req.ErrCh <- err + close(req.ErrCh) + } + + filter.Delete(req) + }() + + } else { + glog.V(4).Infoln("discard volume grow request") + } + } + }() +} + func (ms *MasterServer) LookupVolume(ctx context.Context, req *master_pb.LookupVolumeRequest) (*master_pb.LookupVolumeResponse, error) { if !ms.Topo.IsLeader() { @@ -68,38 +121,45 @@ func (ms *MasterServer) Assign(ctx context.Context, req *master_pb.AssignRequest ReplicaPlacement: replicaPlacement, Ttl: ttl, DiskType: diskType, - Prealloacte: ms.preallocateSize, + Preallocate: ms.preallocateSize, DataCenter: req.DataCenter, Rack: req.Rack, DataNode: req.DataNode, MemoryMapMaxSizeMb: req.MemoryMapMaxSizeMb, } - if !ms.Topo.HasWritableVolume(option) { + if ms.shouldVolumeGrow(option) { if ms.Topo.AvailableSpaceFor(option) <= 0 { return nil, fmt.Errorf("no free volumes left for " + option.String()) } - ms.vgLock.Lock() - if !ms.Topo.HasWritableVolume(option) { - if _, err = ms.vg.AutomaticGrowByType(option, ms.grpcDialOption, ms.Topo, int(req.WritableVolumeCount)); err != nil { - ms.vgLock.Unlock() - return nil, fmt.Errorf("Cannot grow volume group! %v", err) - } + ms.vgCh <- &topology.VolumeGrowRequest{ + Option: option, + Count: int(req.WritableVolumeCount), } - ms.vgLock.Unlock() - } - fid, count, dn, err := ms.Topo.PickForWrite(req.Count, option) - if err != nil { - return nil, fmt.Errorf("%v", err) } - return &master_pb.AssignResponse{ - Fid: fid, - Url: dn.Url(), - PublicUrl: dn.PublicUrl, - Count: count, - Auth: string(security.GenJwt(ms.guard.SigningKey, ms.guard.ExpiresAfterSec, fid)), - }, nil + var ( + lastErr error + maxTimeout = time.Second * 10 + startTime = time.Now() + ) + + for time.Now().Sub(startTime) < maxTimeout { + fid, count, dn, err := ms.Topo.PickForWrite(req.Count, option) + if err == nil { + return &master_pb.AssignResponse{ + Fid: fid, + Url: dn.Url(), + PublicUrl: dn.PublicUrl, + Count: count, + Auth: string(security.GenJwt(ms.guard.SigningKey, ms.guard.ExpiresAfterSec, fid)), + }, nil + } + //glog.V(4).Infoln("waiting for volume growing...") + lastErr = err + time.Sleep(200 * time.Millisecond) + } + return nil, lastErr } func (ms *MasterServer) Statistics(ctx context.Context, req *master_pb.StatisticsRequest) (*master_pb.StatisticsResponse, error) { diff --git a/weed/server/master_server.go b/weed/server/master_server.go index e2b2df18d..838803908 100644 --- a/weed/server/master_server.go +++ b/weed/server/master_server.go @@ -51,9 +51,9 @@ type MasterServer struct { preallocateSize int64 - Topo *topology.Topology - vg *topology.VolumeGrowth - vgLock sync.Mutex + Topo *topology.Topology + vg *topology.VolumeGrowth + vgCh chan *topology.VolumeGrowRequest boundedLeaderChan chan int @@ -82,6 +82,12 @@ func NewMasterServer(r *mux.Router, option *MasterOption, peers []string) *Maste v.SetDefault("master.replication.treat_replication_as_minimums", false) replicationAsMin := v.GetBool("master.replication.treat_replication_as_minimums") + v.SetDefault("master.volume_growth.copy_1", 7) + v.SetDefault("master.volume_growth.copy_2", 6) + v.SetDefault("master.volume_growth.copy_3", 3) + v.SetDefault("master.volume_growth.copy_other", 1) + v.SetDefault("master.volume_growth.threshold", 0.9) + var preallocateSize int64 if option.VolumePreallocate { preallocateSize = int64(option.VolumeSizeLimitMB) * (1 << 20) @@ -91,6 +97,7 @@ func NewMasterServer(r *mux.Router, option *MasterOption, peers []string) *Maste ms := &MasterServer{ option: option, preallocateSize: preallocateSize, + vgCh: make(chan *topology.VolumeGrowRequest, 1 << 6), clientChans: make(map[string]chan *master_pb.VolumeLocation), grpcDialOption: grpcDialOption, MasterClient: wdclient.NewMasterClient(grpcDialOption, "master", option.Host, 0, "", peers), @@ -128,7 +135,14 @@ func NewMasterServer(r *mux.Router, option *MasterOption, peers []string) *Maste r.HandleFunc("/{fileId}", ms.redirectHandler) } - ms.Topo.StartRefreshWritableVolumes(ms.grpcDialOption, ms.option.GarbageThreshold, ms.preallocateSize) + ms.Topo.StartRefreshWritableVolumes( + ms.grpcDialOption, + ms.option.GarbageThreshold, + v.GetFloat64("master.volume_growth.threshold"), + ms.preallocateSize, + ) + + ms.ProcessGrowRequest() ms.startAdminScripts() diff --git a/weed/server/master_server_handlers.go b/weed/server/master_server_handlers.go index a9fecc5bd..974b3308f 100644 --- a/weed/server/master_server_handlers.go +++ b/weed/server/master_server_handlers.go @@ -10,6 +10,7 @@ import ( "github.com/chrislusf/seaweedfs/weed/security" "github.com/chrislusf/seaweedfs/weed/stats" "github.com/chrislusf/seaweedfs/weed/storage/needle" + "github.com/chrislusf/seaweedfs/weed/topology" ) func (ms *MasterServer) lookupVolumeId(vids []string, collection string) (volumeLocations map[string]operation.LookupResult) { @@ -111,19 +112,20 @@ func (ms *MasterServer) dirAssignHandler(w http.ResponseWriter, r *http.Request) return } - if !ms.Topo.HasWritableVolume(option) { + if ms.shouldVolumeGrow(option) { if ms.Topo.AvailableSpaceFor(option) <= 0 { writeJsonQuiet(w, r, http.StatusNotFound, operation.AssignResult{Error: "No free volumes left for " + option.String()}) return } - ms.vgLock.Lock() - defer ms.vgLock.Unlock() - if !ms.Topo.HasWritableVolume(option) { - if _, err = ms.vg.AutomaticGrowByType(option, ms.grpcDialOption, ms.Topo, writableVolumeCount); err != nil { - writeJsonError(w, r, http.StatusInternalServerError, - fmt.Errorf("Cannot grow volume group! %v", err)) - return - } + errCh := make(chan error, 1) + ms.vgCh <- &topology.VolumeGrowRequest{ + Option: option, + Count: writableVolumeCount, + ErrCh: errCh, + } + if err := <- errCh; err != nil { + writeJsonError(w, r, http.StatusInternalServerError, fmt.Errorf("cannot grow volume group! %v", err)) + return } } fid, count, dn, err := ms.Topo.PickForWrite(requestedCount, option) diff --git a/weed/server/master_server_handlers_admin.go b/weed/server/master_server_handlers_admin.go index f24d4e924..fb16ef78c 100644 --- a/weed/server/master_server_handlers_admin.go +++ b/weed/server/master_server_handlers_admin.go @@ -3,7 +3,6 @@ package weed_server import ( "context" "fmt" - "github.com/chrislusf/seaweedfs/weed/storage/types" "math/rand" "net/http" "strconv" @@ -14,6 +13,7 @@ import ( "github.com/chrislusf/seaweedfs/weed/storage/backend/memory_map" "github.com/chrislusf/seaweedfs/weed/storage/needle" "github.com/chrislusf/seaweedfs/weed/storage/super_block" + "github.com/chrislusf/seaweedfs/weed/storage/types" "github.com/chrislusf/seaweedfs/weed/topology" "github.com/chrislusf/seaweedfs/weed/util" ) @@ -136,9 +136,11 @@ func (ms *MasterServer) submitFromMasterServerHandler(w http.ResponseWriter, r * } } -func (ms *MasterServer) HasWritableVolume(option *topology.VolumeGrowOption) bool { +func (ms *MasterServer) shouldVolumeGrow(option *topology.VolumeGrowOption) bool { vl := ms.Topo.GetVolumeLayout(option.Collection, option.ReplicaPlacement, option.Ttl, option.DiskType) - return vl.GetActiveVolumeCount(option) > 0 + active, high := vl.GetActiveVolumeCount(option) + //glog.V(0).Infof("active volume: %d, high usage volume: %d\n", active, high) + return active <= high } func (ms *MasterServer) getVolumeGrowOption(r *http.Request) (*topology.VolumeGrowOption, error) { @@ -172,7 +174,7 @@ func (ms *MasterServer) getVolumeGrowOption(r *http.Request) (*topology.VolumeGr ReplicaPlacement: replicaPlacement, Ttl: ttl, DiskType: diskType, - Prealloacte: preallocate, + Preallocate: preallocate, DataCenter: r.FormValue("dataCenter"), Rack: r.FormValue("rack"), DataNode: r.FormValue("dataNode"), diff --git a/weed/topology/allocate_volume.go b/weed/topology/allocate_volume.go index 39c24ab04..7c7fae683 100644 --- a/weed/topology/allocate_volume.go +++ b/weed/topology/allocate_volume.go @@ -22,7 +22,7 @@ func AllocateVolume(dn *DataNode, grpcDialOption grpc.DialOption, vid needle.Vol Collection: option.Collection, Replication: option.ReplicaPlacement.String(), Ttl: option.Ttl.String(), - Preallocate: option.Prealloacte, + Preallocate: option.Preallocate, MemoryMapMaxSizeMb: option.MemoryMapMaxSizeMb, DiskType: string(option.DiskType), }) diff --git a/weed/topology/node.go b/weed/topology/node.go index 95d63972e..a23729dd3 100644 --- a/weed/topology/node.go +++ b/weed/topology/node.go @@ -25,7 +25,7 @@ type Node interface { SetParent(Node) LinkChildNode(node Node) UnlinkChildNode(nodeId NodeId) - CollectDeadNodeAndFullVolumes(freshThreshHold int64, volumeSizeLimit uint64) + CollectDeadNodeAndFullVolumes(freshThreshHold int64, volumeSizeLimit uint64, growThreshold float64) IsDataNode() bool IsRack() bool @@ -235,20 +235,22 @@ func (n *NodeImpl) UnlinkChildNode(nodeId NodeId) { } } -func (n *NodeImpl) CollectDeadNodeAndFullVolumes(freshThreshHold int64, volumeSizeLimit uint64) { +func (n *NodeImpl) CollectDeadNodeAndFullVolumes(freshThreshHold int64, volumeSizeLimit uint64, growThreshold float64) { if n.IsRack() { for _, c := range n.Children() { dn := c.(*DataNode) //can not cast n to DataNode for _, v := range dn.GetVolumes() { - if uint64(v.Size) >= volumeSizeLimit { + if v.Size >= volumeSizeLimit { //fmt.Println("volume",v.Id,"size",v.Size,">",volumeSizeLimit) - n.GetTopology().chanFullVolumes <- v + n.GetTopology().chanFullVolumes <- &v + }else if float64(v.Size) > float64(volumeSizeLimit) * growThreshold { + n.GetTopology().chanCrowdedVolumes <- &v } } } } else { for _, c := range n.Children() { - c.CollectDeadNodeAndFullVolumes(freshThreshHold, volumeSizeLimit) + c.CollectDeadNodeAndFullVolumes(freshThreshHold, volumeSizeLimit, growThreshold) } } } diff --git a/weed/topology/topology.go b/weed/topology/topology.go index 08ebd24fd..3932e3fbb 100644 --- a/weed/topology/topology.go +++ b/weed/topology/topology.go @@ -34,9 +34,10 @@ type Topology struct { Sequence sequence.Sequencer - chanFullVolumes chan storage.VolumeInfo + chanFullVolumes chan *storage.VolumeInfo + chanCrowdedVolumes chan *storage.VolumeInfo - Configuration *Configuration + Configuration *Configuration RaftServer raft.Server } @@ -56,7 +57,8 @@ func NewTopology(id string, seq sequence.Sequencer, volumeSizeLimit uint64, puls t.Sequence = seq - t.chanFullVolumes = make(chan storage.VolumeInfo) + t.chanFullVolumes = make(chan *storage.VolumeInfo) + t.chanCrowdedVolumes = make(chan *storage.VolumeInfo) t.Configuration = &Configuration{} @@ -122,9 +124,11 @@ func (t *Topology) NextVolumeId() (needle.VolumeId, error) { return next, nil } +// deprecated func (t *Topology) HasWritableVolume(option *VolumeGrowOption) bool { vl := t.GetVolumeLayout(option.Collection, option.ReplicaPlacement, option.Ttl, option.DiskType) - return vl.GetActiveVolumeCount(option) > 0 + active, _ := vl.GetActiveVolumeCount(option) + return active > 0 } func (t *Topology) PickForWrite(count uint64, option *VolumeGrowOption) (string, uint64, *DataNode, error) { diff --git a/weed/topology/topology_event_handling.go b/weed/topology/topology_event_handling.go index 543dacf29..e4eb430fe 100644 --- a/weed/topology/topology_event_handling.go +++ b/weed/topology/topology_event_handling.go @@ -10,12 +10,12 @@ import ( "github.com/chrislusf/seaweedfs/weed/storage" ) -func (t *Topology) StartRefreshWritableVolumes(grpcDialOption grpc.DialOption, garbageThreshold float64, preallocate int64) { +func (t *Topology) StartRefreshWritableVolumes(grpcDialOption grpc.DialOption, garbageThreshold float64, growThreshold float64, preallocate int64) { go func() { for { if t.IsLeader() { freshThreshHold := time.Now().Unix() - 3*t.pulse //3 times of sleep interval - t.CollectDeadNodeAndFullVolumes(freshThreshHold, t.volumeSizeLimit) + t.CollectDeadNodeAndFullVolumes(freshThreshHold, t.volumeSizeLimit, growThreshold) } time.Sleep(time.Duration(float32(t.pulse*1e3)*(1+rand.Float32())) * time.Millisecond) } @@ -31,13 +31,15 @@ func (t *Topology) StartRefreshWritableVolumes(grpcDialOption grpc.DialOption, g go func() { for { select { - case v := <-t.chanFullVolumes: - t.SetVolumeCapacityFull(v) + case fv := <-t.chanFullVolumes: + t.SetVolumeCapacityFull(fv) + case cv := <-t.chanCrowdedVolumes: + t.SetVolumeCrowded(cv) } } }() } -func (t *Topology) SetVolumeCapacityFull(volumeInfo storage.VolumeInfo) bool { +func (t *Topology) SetVolumeCapacityFull(volumeInfo *storage.VolumeInfo) bool { diskType := types.ToDiskType(volumeInfo.DiskType) vl := t.GetVolumeLayout(volumeInfo.Collection, volumeInfo.ReplicaPlacement, volumeInfo.Ttl, diskType) if !vl.SetVolumeCapacityFull(volumeInfo.Id) { @@ -60,6 +62,13 @@ func (t *Topology) SetVolumeCapacityFull(volumeInfo storage.VolumeInfo) bool { } return true } + +func (t *Topology) SetVolumeCrowded(volumeInfo *storage.VolumeInfo) { + diskType := types.ToDiskType(volumeInfo.DiskType) + vl := t.GetVolumeLayout(volumeInfo.Collection, volumeInfo.ReplicaPlacement, volumeInfo.Ttl, diskType) + vl.SetVolumeCrowded(volumeInfo.Id) +} + func (t *Topology) UnRegisterDataNode(dn *DataNode) { for _, v := range dn.GetVolumes() { glog.V(0).Infoln("Removing Volume", v.Id, "from the dead volume server", dn.Id()) diff --git a/weed/topology/volume_growth.go b/weed/topology/volume_growth.go index 8941a049b..ae0b11c81 100644 --- a/weed/topology/volume_growth.go +++ b/weed/topology/volume_growth.go @@ -3,18 +3,17 @@ package topology import ( "encoding/json" "fmt" - "github.com/chrislusf/seaweedfs/weed/storage/types" "math/rand" "sync" - "github.com/chrislusf/seaweedfs/weed/storage/needle" - "github.com/chrislusf/seaweedfs/weed/storage/super_block" - "github.com/chrislusf/seaweedfs/weed/util" - "google.golang.org/grpc" "github.com/chrislusf/seaweedfs/weed/glog" "github.com/chrislusf/seaweedfs/weed/storage" + "github.com/chrislusf/seaweedfs/weed/storage/needle" + "github.com/chrislusf/seaweedfs/weed/storage/super_block" + "github.com/chrislusf/seaweedfs/weed/storage/types" + "github.com/chrislusf/seaweedfs/weed/util" ) /* @@ -25,12 +24,18 @@ This package is created to resolve these replica placement issues: 4. volume allocation for each bucket */ +type VolumeGrowRequest struct { + Option *VolumeGrowOption + Count int + ErrCh chan error +} + type VolumeGrowOption struct { Collection string `json:"collection,omitempty"` ReplicaPlacement *super_block.ReplicaPlacement `json:"replication,omitempty"` Ttl *needle.TTL `json:"ttl,omitempty"` DiskType types.DiskType `json:"disk,omitempty"` - Prealloacte int64 `json:"prealloacte,omitempty"` + Preallocate int64 `json:"preallocate,omitempty"` DataCenter string `json:"dataCenter,omitempty"` Rack string `json:"rack,omitempty"` DataNode string `json:"dataNode,omitempty"` @@ -46,6 +51,11 @@ func (o *VolumeGrowOption) String() string { return string(blob) } +func (o *VolumeGrowOption) Threshold() float64 { + v := util.GetViper() + return v.GetFloat64("master.volume_growth.threshold") +} + func NewDefaultVolumeGrowth() *VolumeGrowth { return &VolumeGrowth{} } @@ -54,10 +64,6 @@ func NewDefaultVolumeGrowth() *VolumeGrowth { // given copyCount, how many logical volumes to create func (vg *VolumeGrowth) findVolumeCount(copyCount int) (count int) { v := util.GetViper() - v.SetDefault("master.volume_growth.copy_1", 7) - v.SetDefault("master.volume_growth.copy_2", 6) - v.SetDefault("master.volume_growth.copy_3", 3) - v.SetDefault("master.volume_growth.copy_other", 1) switch copyCount { case 1: count = v.GetInt("master.volume_growth.copy_1") diff --git a/weed/topology/volume_layout.go b/weed/topology/volume_layout.go index c7e171248..57e511fa0 100644 --- a/weed/topology/volume_layout.go +++ b/weed/topology/volume_layout.go @@ -27,6 +27,7 @@ type volumeState string const ( readOnlyState volumeState = "ReadOnly" oversizedState = "Oversized" + crowdedState = "Crowded" ) type stateIndicator func(copyState) bool @@ -106,7 +107,8 @@ type VolumeLayout struct { ttl *needle.TTL diskType types.DiskType vid2location map[needle.VolumeId]*VolumeLocationList - writables []needle.VolumeId // transient array of writable volume id + writables []needle.VolumeId // transient array of writable volume id + crowded map[needle.VolumeId]interface{} readonlyVolumes *volumesBinaryState // readonly volumes oversizedVolumes *volumesBinaryState // oversized volumes volumeSizeLimit uint64 @@ -127,6 +129,7 @@ func NewVolumeLayout(rp *super_block.ReplicaPlacement, ttl *needle.TTL, diskType diskType: diskType, vid2location: make(map[needle.VolumeId]*VolumeLocationList), writables: *new([]needle.VolumeId), + crowded: make(map[needle.VolumeId]interface{}), readonlyVolumes: NewVolumesBinaryState(readOnlyState, rp, ExistCopies()), oversizedVolumes: NewVolumesBinaryState(oversizedState, rp, ExistCopies()), volumeSizeLimit: volumeSizeLimit, @@ -273,7 +276,7 @@ func (vl *VolumeLayout) PickForWrite(count uint64, option *VolumeGrowOption) (*n lenWriters := len(vl.writables) if lenWriters <= 0 { - glog.V(0).Infoln("No more writable volumes!") + //glog.V(0).Infoln("No more writable volumes!") return nil, 0, nil, errors.New("No more writable volumes!") } if option.DataCenter == "" { @@ -307,14 +310,13 @@ func (vl *VolumeLayout) PickForWrite(count uint64, option *VolumeGrowOption) (*n return &vid, count, locationList, nil } -func (vl *VolumeLayout) GetActiveVolumeCount(option *VolumeGrowOption) int { +func (vl *VolumeLayout) GetActiveVolumeCount(option *VolumeGrowOption) (active, crowded int) { vl.accessLock.RLock() defer vl.accessLock.RUnlock() if option.DataCenter == "" { - return len(vl.writables) + return len(vl.writables), len(vl.crowded) } - counter := 0 for _, v := range vl.writables { for _, dn := range vl.vid2location[v].list { if dn.GetDataCenter().Id() == NodeId(option.DataCenter) { @@ -324,11 +326,15 @@ func (vl *VolumeLayout) GetActiveVolumeCount(option *VolumeGrowOption) int { if option.DataNode != "" && dn.Id() != NodeId(option.DataNode) { continue } - counter++ + active++ + info, _ := dn.GetVolumesById(v) + if float64(info.Size) > float64(vl.volumeSizeLimit)*option.Threshold() { + crowded++ + } } } } - return counter + return } func (vl *VolumeLayout) removeFromWritable(vid needle.VolumeId) bool { @@ -342,6 +348,7 @@ func (vl *VolumeLayout) removeFromWritable(vid needle.VolumeId) bool { if toDeleteIndex >= 0 { glog.V(0).Infoln("Volume", vid, "becomes unwritable") vl.writables = append(vl.writables[0:toDeleteIndex], vl.writables[toDeleteIndex+1:]...) + vl.removeFromCrowded(vid) return true } return false @@ -408,6 +415,32 @@ func (vl *VolumeLayout) SetVolumeCapacityFull(vid needle.VolumeId) bool { return vl.removeFromWritable(vid) } +func (vl *VolumeLayout) removeFromCrowded(vid needle.VolumeId) { + delete(vl.crowded, vid) +} + +func (vl *VolumeLayout) setVolumeCrowded(vid needle.VolumeId) { + if _, ok := vl.crowded[vid]; !ok { + vl.crowded[vid] = nil + glog.V(0).Infoln("Volume", vid, "becomes crowded") + } +} + +func (vl *VolumeLayout) SetVolumeCrowded(vid needle.VolumeId) { + // since delete is guarded by accessLock.Lock(), + // and is always called in sequential order, + // RLock() should be safe enough + vl.accessLock.RLock() + defer vl.accessLock.RUnlock() + + for _, v := range vl.writables { + if v == vid { + vl.setVolumeCrowded(vid) + break + } + } +} + func (vl *VolumeLayout) ToMap() map[string]interface{} { m := make(map[string]interface{}) m["replication"] = vl.rp.String()