aboutsummaryrefslogtreecommitdiff
path: root/weed/shell/command_volume_tier_move.go
diff options
context:
space:
mode:
authorBl1tz23 <alex3angle@gmail.com>2021-08-10 13:45:24 +0300
committerBl1tz23 <alex3angle@gmail.com>2021-08-10 13:45:24 +0300
commit1c94b3d01340baad000188550fcf2ccab6ca80e5 (patch)
tree12c3da17eb2d1a43fef78021a3d7c79110b0ff5f /weed/shell/command_volume_tier_move.go
parente6e57db530217ff57b3622b4672b03ebb6313e96 (diff)
parentf9cf9b93d32a2b01bc4d95ce7d24d86ef60be668 (diff)
downloadseaweedfs-1c94b3d01340baad000188550fcf2ccab6ca80e5.tar.xz
seaweedfs-1c94b3d01340baad000188550fcf2ccab6ca80e5.zip
merge master, resolve conflicts
Diffstat (limited to 'weed/shell/command_volume_tier_move.go')
-rw-r--r--weed/shell/command_volume_tier_move.go90
1 files changed, 70 insertions, 20 deletions
diff --git a/weed/shell/command_volume_tier_move.go b/weed/shell/command_volume_tier_move.go
index d6a49d6e1..d370d93e4 100644
--- a/weed/shell/command_volume_tier_move.go
+++ b/weed/shell/command_volume_tier_move.go
@@ -7,6 +7,8 @@ import (
"github.com/chrislusf/seaweedfs/weed/storage/types"
"github.com/chrislusf/seaweedfs/weed/wdclient"
"io"
+ "path/filepath"
+ "sync"
"time"
"github.com/chrislusf/seaweedfs/weed/storage/needle"
@@ -17,6 +19,9 @@ func init() {
}
type commandVolumeTierMove struct {
+ activeServers map[string]struct{}
+ activeServersLock sync.Mutex
+ activeServersCond *sync.Cond
}
func (c *commandVolumeTierMove) Name() string {
@@ -26,7 +31,7 @@ func (c *commandVolumeTierMove) Name() string {
func (c *commandVolumeTierMove) Help() string {
return `change a volume from one disk type to another
- volume.tier.move -fromDiskType=hdd -toDiskType=ssd [-collection=""] [-fullPercent=95] [-quietFor=1h]
+ volume.tier.move -fromDiskType=hdd -toDiskType=ssd [-collectionPattern=""] [-fullPercent=95] [-quietFor=1h]
Even if the volume is replicated, only one replica will be changed and the rest replicas will be dropped.
So "volume.fix.replication" and "volume.balance" should be followed.
@@ -36,12 +41,15 @@ func (c *commandVolumeTierMove) Help() string {
func (c *commandVolumeTierMove) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
+ c.activeServers = make(map[string]struct{})
+ c.activeServersCond = sync.NewCond(new(sync.Mutex))
+
if err = commandEnv.confirmIsLocked(); err != nil {
return
}
tierCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
- collection := tierCommand.String("collection", "", "the collection name")
+ collectionPattern := tierCommand.String("collectionPattern", "", "match with wildcard characters '*' and '?'")
fullPercentage := tierCommand.Float64("fullPercent", 95, "the volume reaches the percentage of max volume size")
quietPeriod := tierCommand.Duration("quietFor", 24*time.Hour, "select volumes without no writes for this period")
source := tierCommand.String("fromDiskType", "", "the source disk type")
@@ -65,7 +73,7 @@ func (c *commandVolumeTierMove) Do(args []string, commandEnv *CommandEnv, writer
}
// collect all volumes that should change
- volumeIds, err := collectVolumeIdsForTierChange(commandEnv, topologyInfo, volumeSizeLimitMb, fromDiskType, *collection, *fullPercentage, *quietPeriod)
+ volumeIds, err := collectVolumeIdsForTierChange(commandEnv, topologyInfo, volumeSizeLimitMb, fromDiskType, *collectionPattern, *fullPercentage, *quietPeriod)
if err != nil {
return err
}
@@ -73,7 +81,7 @@ func (c *commandVolumeTierMove) Do(args []string, commandEnv *CommandEnv, writer
_, allLocations := collectVolumeReplicaLocations(topologyInfo)
for _, vid := range volumeIds {
- if err = doVolumeTierMove(commandEnv, writer, *collection, vid, toDiskType, allLocations, *applyChange); err != nil {
+ if err = c.doVolumeTierMove(commandEnv, writer, vid, toDiskType, allLocations, *applyChange); err != nil {
fmt.Printf("tier move volume %d: %v\n", vid, err)
}
}
@@ -90,7 +98,7 @@ func isOneOf(server string, locations []wdclient.Location) bool {
return false
}
-func doVolumeTierMove(commandEnv *CommandEnv, writer io.Writer, collection string, vid needle.VolumeId, toDiskType types.DiskType, allLocations []location, applyChanges bool) (err error) {
+func (c *commandVolumeTierMove) doVolumeTierMove(commandEnv *CommandEnv, writer io.Writer, vid needle.VolumeId, toDiskType types.DiskType, allLocations []location, applyChanges bool) (err error) {
// find volume location
locations, found := commandEnv.MasterClient.GetLocations(uint32(vid))
if !found {
@@ -120,25 +128,32 @@ func doVolumeTierMove(commandEnv *CommandEnv, writer io.Writer, collection strin
hasFoundTarget = true
if !applyChanges {
+ // adjust volume count
+ dst.dataNode.DiskInfos[string(toDiskType)].VolumeCount++
break
}
- // mark all replicas as read only
- if err = markVolumeReadonly(commandEnv.option.GrpcDialOption, vid, locations); err != nil {
- return fmt.Errorf("mark volume %d as readonly on %s: %v", vid, locations[0].Url, err)
- }
- if err = LiveMoveVolume(commandEnv.option.GrpcDialOption, vid, sourceVolumeServer, dst.dataNode.Id, 5*time.Second, toDiskType.ReadableString()); err != nil {
- return fmt.Errorf("move volume %d %s => %s : %v", vid, locations[0].Url, dst.dataNode.Id, err)
+ c.activeServersCond.L.Lock()
+ _, isSourceActive := c.activeServers[sourceVolumeServer]
+ _, isDestActive := c.activeServers[dst.dataNode.Id]
+ for isSourceActive || isDestActive {
+ c.activeServersCond.Wait()
+ _, isSourceActive = c.activeServers[sourceVolumeServer]
+ _, isDestActive = c.activeServers[dst.dataNode.Id]
}
+ c.activeServers[sourceVolumeServer] = struct{}{}
+ c.activeServers[dst.dataNode.Id] = struct{}{}
+ c.activeServersCond.L.Unlock()
- // remove the remaining replicas
- for _, loc := range locations {
- if loc.Url != dst.dataNode.Id {
- if err = deleteVolume(commandEnv.option.GrpcDialOption, vid, loc.Url); err != nil {
- fmt.Fprintf(writer, "failed to delete volume %d on %s\n", vid, loc.Url)
- }
+ go func(dst location) {
+ if err := c.doMoveOneVolume(commandEnv, writer, vid, toDiskType, locations, sourceVolumeServer, dst); err != nil {
+ fmt.Fprintf(writer, "move volume %d %s => %s: %v\n", vid, sourceVolumeServer, dst.dataNode.Id, err)
}
- }
+ delete(c.activeServers, sourceVolumeServer)
+ delete(c.activeServers, dst.dataNode.Id)
+ c.activeServersCond.Signal()
+ }(dst)
+
}
}
@@ -149,7 +164,31 @@ func doVolumeTierMove(commandEnv *CommandEnv, writer io.Writer, collection strin
return nil
}
-func collectVolumeIdsForTierChange(commandEnv *CommandEnv, topologyInfo *master_pb.TopologyInfo, volumeSizeLimitMb uint64, sourceTier types.DiskType, selectedCollection string, fullPercentage float64, quietPeriod time.Duration) (vids []needle.VolumeId, err error) {
+func (c *commandVolumeTierMove) doMoveOneVolume(commandEnv *CommandEnv, writer io.Writer, vid needle.VolumeId, toDiskType types.DiskType, locations []wdclient.Location, sourceVolumeServer string, dst location) (err error) {
+
+ // mark all replicas as read only
+ if err = markVolumeReadonly(commandEnv.option.GrpcDialOption, vid, locations); err != nil {
+ return fmt.Errorf("mark volume %d as readonly on %s: %v", vid, locations[0].Url, err)
+ }
+ if err = LiveMoveVolume(commandEnv.option.GrpcDialOption, writer, vid, sourceVolumeServer, dst.dataNode.Id, 5*time.Second, toDiskType.ReadableString(), true); err != nil {
+ return fmt.Errorf("move volume %d %s => %s : %v", vid, locations[0].Url, dst.dataNode.Id, err)
+ }
+
+ // adjust volume count
+ dst.dataNode.DiskInfos[string(toDiskType)].VolumeCount++
+
+ // remove the remaining replicas
+ for _, loc := range locations {
+ if loc.Url != dst.dataNode.Id && loc.Url != sourceVolumeServer {
+ if err = deleteVolume(commandEnv.option.GrpcDialOption, vid, loc.Url); err != nil {
+ fmt.Fprintf(writer, "failed to delete volume %d on %s: %v\n", vid, loc.Url, err)
+ }
+ }
+ }
+ return nil
+}
+
+func collectVolumeIdsForTierChange(commandEnv *CommandEnv, topologyInfo *master_pb.TopologyInfo, volumeSizeLimitMb uint64, sourceTier types.DiskType, collectionPattern string, fullPercentage float64, quietPeriod time.Duration) (vids []needle.VolumeId, err error) {
quietSeconds := int64(quietPeriod / time.Second)
nowUnixSeconds := time.Now().Unix()
@@ -160,7 +199,18 @@ func collectVolumeIdsForTierChange(commandEnv *CommandEnv, topologyInfo *master_
eachDataNode(topologyInfo, func(dc string, rack RackId, dn *master_pb.DataNodeInfo) {
for _, diskInfo := range dn.DiskInfos {
for _, v := range diskInfo.VolumeInfos {
- if v.Collection == selectedCollection && v.ModifiedAtSecond+quietSeconds < nowUnixSeconds && types.ToDiskType(v.DiskType) == sourceTier {
+ // check collection name pattern
+ if collectionPattern != "" {
+ matched, err := filepath.Match(collectionPattern, v.Collection)
+ if err != nil {
+ return
+ }
+ if !matched {
+ continue
+ }
+ }
+
+ if v.ModifiedAtSecond+quietSeconds < nowUnixSeconds && types.ToDiskType(v.DiskType) == sourceTier {
if float64(v.Size) > fullPercentage/100*float64(volumeSizeLimitMb)*1024*1024 {
vidMap[v.Id] = true
}