mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2024-01-19 02:48:24 +00:00
ahead of time volume assignment
This commit is contained in:
parent
c48ef78670
commit
c4d32f6937
|
@ -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) {
|
||||
|
|
|
@ -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()
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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"),
|
||||
|
|
|
@ -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),
|
||||
})
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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()
|
||||
|
|
Loading…
Reference in a new issue