aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorqieqieplus <admin@qieqie.me>2021-05-06 18:46:14 +0800
committerqieqieplus <admin@qieqie.me>2021-05-06 18:55:44 +0800
commitc4d32f6937236a83377e3937e311222fc04bd084 (patch)
tree6d803bc4df6d7d889b8ca883d5ad5181502ebe5f
parentc48ef786702e4c62f644631d37d31218f7447fe4 (diff)
downloadseaweedfs-c4d32f6937236a83377e3937e311222fc04bd084.tar.xz
seaweedfs-c4d32f6937236a83377e3937e311222fc04bd084.zip
ahead of time volume assignment
-rw-r--r--weed/server/master_grpc_server_volume.go102
-rw-r--r--weed/server/master_server.go22
-rw-r--r--weed/server/master_server_handlers.go20
-rw-r--r--weed/server/master_server_handlers_admin.go10
-rw-r--r--weed/topology/allocate_volume.go2
-rw-r--r--weed/topology/node.go12
-rw-r--r--weed/topology/topology.go12
-rw-r--r--weed/topology/topology_event_handling.go19
-rw-r--r--weed/topology/volume_growth.go26
-rw-r--r--weed/topology/volume_layout.go47
10 files changed, 202 insertions, 70 deletions
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()