master_grpc_server_volume.go 11 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373
  1. package weed_server
  2. import (
  3. "context"
  4. "fmt"
  5. "math"
  6. "math/rand/v2"
  7. "strings"
  8. "sync"
  9. "time"
  10. "github.com/seaweedfs/seaweedfs/weed/stats"
  11. "github.com/seaweedfs/seaweedfs/weed/topology"
  12. "github.com/seaweedfs/raft"
  13. "github.com/seaweedfs/seaweedfs/weed/glog"
  14. "github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
  15. "github.com/seaweedfs/seaweedfs/weed/security"
  16. "github.com/seaweedfs/seaweedfs/weed/storage/needle"
  17. "github.com/seaweedfs/seaweedfs/weed/storage/super_block"
  18. "github.com/seaweedfs/seaweedfs/weed/storage/types"
  19. )
  20. const (
  21. volumeGrowStepCount = 2
  22. )
  23. func (ms *MasterServer) DoAutomaticVolumeGrow(req *topology.VolumeGrowRequest) {
  24. if ms.option.VolumeGrowthDisabled {
  25. glog.V(1).Infof("automatic volume grow disabled")
  26. return
  27. }
  28. glog.V(1).Infoln("starting automatic volume grow")
  29. start := time.Now()
  30. newVidLocations, err := ms.vg.AutomaticGrowByType(req.Option, ms.grpcDialOption, ms.Topo, req.Count)
  31. glog.V(1).Infoln("finished automatic volume grow, cost ", time.Now().Sub(start))
  32. if err != nil {
  33. glog.V(1).Infof("automatic volume grow failed: %+v", err)
  34. return
  35. }
  36. for _, newVidLocation := range newVidLocations {
  37. ms.broadcastToClients(&master_pb.KeepConnectedResponse{VolumeLocation: newVidLocation})
  38. }
  39. }
  40. func (ms *MasterServer) ProcessGrowRequest() {
  41. go func() {
  42. ctx := context.Background()
  43. firstRun := true
  44. for {
  45. if firstRun {
  46. firstRun = false
  47. } else {
  48. time.Sleep(5*time.Minute + time.Duration(30*rand.Float32())*time.Second)
  49. }
  50. if !ms.Topo.IsLeader() {
  51. continue
  52. }
  53. dcs := ms.Topo.ListDCAndRacks()
  54. var err error
  55. for _, vlc := range ms.Topo.ListVolumeLayoutCollections() {
  56. vl := vlc.VolumeLayout
  57. lastGrowCount := vl.GetLastGrowCount()
  58. if vl.HasGrowRequest() {
  59. continue
  60. }
  61. writable, crowded := vl.GetWritableVolumeCount()
  62. mustGrow := int(lastGrowCount) - writable
  63. vgr := vlc.ToVolumeGrowRequest()
  64. stats.MasterVolumeLayoutWritable.WithLabelValues(vlc.Collection, vgr.DiskType, vgr.Replication, vgr.Ttl).Set(float64(writable))
  65. stats.MasterVolumeLayoutCrowded.WithLabelValues(vlc.Collection, vgr.DiskType, vgr.Replication, vgr.Ttl).Set(float64(crowded))
  66. switch {
  67. case mustGrow > 0:
  68. vgr.WritableVolumeCount = uint32(mustGrow)
  69. _, err = ms.VolumeGrow(ctx, vgr)
  70. case lastGrowCount > 0 && writable < int(lastGrowCount*2) && float64(crowded+volumeGrowStepCount) > float64(writable)*topology.VolumeGrowStrategy.Threshold:
  71. vgr.WritableVolumeCount = volumeGrowStepCount
  72. _, err = ms.VolumeGrow(ctx, vgr)
  73. }
  74. if err != nil {
  75. glog.V(0).Infof("volume grow request failed: %+v", err)
  76. }
  77. writableVolumes := vl.CloneWritableVolumes()
  78. for dcId, racks := range dcs {
  79. for _, rackId := range racks {
  80. if vl.ShouldGrowVolumesByDcAndRack(&writableVolumes, dcId, rackId) {
  81. vgr.DataCenter = string(dcId)
  82. vgr.Rack = string(rackId)
  83. if lastGrowCount > 0 {
  84. vgr.WritableVolumeCount = uint32(math.Ceil(float64(lastGrowCount) / float64(len(dcs)*len(racks))))
  85. } else {
  86. vgr.WritableVolumeCount = volumeGrowStepCount
  87. }
  88. if _, err = ms.VolumeGrow(ctx, vgr); err != nil {
  89. glog.V(0).Infof("volume grow request for dc:%s rack:%s failed: %+v", dcId, rackId, err)
  90. }
  91. }
  92. }
  93. }
  94. }
  95. }
  96. }()
  97. go func() {
  98. filter := sync.Map{}
  99. for {
  100. req, ok := <-ms.volumeGrowthRequestChan
  101. if !ok {
  102. break
  103. }
  104. option := req.Option
  105. vl := ms.Topo.GetVolumeLayout(option.Collection, option.ReplicaPlacement, option.Ttl, option.DiskType)
  106. if !ms.Topo.IsLeader() {
  107. //discard buffered requests
  108. time.Sleep(time.Second * 1)
  109. vl.DoneGrowRequest()
  110. continue
  111. }
  112. // filter out identical requests being processed
  113. found := false
  114. filter.Range(func(k, v interface{}) bool {
  115. existingReq := k.(*topology.VolumeGrowRequest)
  116. if existingReq.Equals(req) {
  117. found = true
  118. }
  119. return !found
  120. })
  121. // not atomic but it's okay
  122. if found || (!req.Force && !vl.ShouldGrowVolumes()) {
  123. glog.V(4).Infoln("discard volume grow request")
  124. time.Sleep(time.Millisecond * 211)
  125. vl.DoneGrowRequest()
  126. continue
  127. }
  128. filter.Store(req, nil)
  129. // we have lock called inside vg
  130. glog.V(0).Infof("volume grow %+v", req)
  131. go func(req *topology.VolumeGrowRequest, vl *topology.VolumeLayout) {
  132. ms.DoAutomaticVolumeGrow(req)
  133. vl.DoneGrowRequest()
  134. filter.Delete(req)
  135. }(req, vl)
  136. }
  137. }()
  138. }
  139. func (ms *MasterServer) LookupVolume(ctx context.Context, req *master_pb.LookupVolumeRequest) (*master_pb.LookupVolumeResponse, error) {
  140. resp := &master_pb.LookupVolumeResponse{}
  141. volumeLocations := ms.lookupVolumeId(req.VolumeOrFileIds, req.Collection)
  142. for _, volumeOrFileId := range req.VolumeOrFileIds {
  143. vid := volumeOrFileId
  144. commaSep := strings.Index(vid, ",")
  145. if commaSep > 0 {
  146. vid = vid[0:commaSep]
  147. }
  148. if result, found := volumeLocations[vid]; found {
  149. var locations []*master_pb.Location
  150. for _, loc := range result.Locations {
  151. locations = append(locations, &master_pb.Location{
  152. Url: loc.Url,
  153. PublicUrl: loc.PublicUrl,
  154. DataCenter: loc.DataCenter,
  155. GrpcPort: uint32(loc.GrpcPort),
  156. })
  157. }
  158. var auth string
  159. if commaSep > 0 { // this is a file id
  160. auth = string(security.GenJwtForVolumeServer(ms.guard.SigningKey, ms.guard.ExpiresAfterSec, result.VolumeOrFileId))
  161. }
  162. resp.VolumeIdLocations = append(resp.VolumeIdLocations, &master_pb.LookupVolumeResponse_VolumeIdLocation{
  163. VolumeOrFileId: result.VolumeOrFileId,
  164. Locations: locations,
  165. Error: result.Error,
  166. Auth: auth,
  167. })
  168. }
  169. }
  170. return resp, nil
  171. }
  172. func (ms *MasterServer) Statistics(ctx context.Context, req *master_pb.StatisticsRequest) (*master_pb.StatisticsResponse, error) {
  173. if !ms.Topo.IsLeader() {
  174. return nil, raft.NotLeaderError
  175. }
  176. if req.Replication == "" {
  177. req.Replication = ms.option.DefaultReplicaPlacement
  178. }
  179. replicaPlacement, err := super_block.NewReplicaPlacementFromString(req.Replication)
  180. if err != nil {
  181. return nil, err
  182. }
  183. ttl, err := needle.ReadTTL(req.Ttl)
  184. if err != nil {
  185. return nil, err
  186. }
  187. volumeLayout := ms.Topo.GetVolumeLayout(req.Collection, replicaPlacement, ttl, types.ToDiskType(req.DiskType))
  188. stats := volumeLayout.Stats()
  189. totalSize := ms.Topo.GetDiskUsages().GetMaxVolumeCount() * int64(ms.option.VolumeSizeLimitMB) * 1024 * 1024
  190. resp := &master_pb.StatisticsResponse{
  191. TotalSize: uint64(totalSize),
  192. UsedSize: stats.UsedSize,
  193. FileCount: stats.FileCount,
  194. }
  195. return resp, nil
  196. }
  197. func (ms *MasterServer) VolumeList(ctx context.Context, req *master_pb.VolumeListRequest) (*master_pb.VolumeListResponse, error) {
  198. if !ms.Topo.IsLeader() {
  199. return nil, raft.NotLeaderError
  200. }
  201. resp := &master_pb.VolumeListResponse{
  202. TopologyInfo: ms.Topo.ToTopologyInfo(),
  203. VolumeSizeLimitMb: uint64(ms.option.VolumeSizeLimitMB),
  204. }
  205. return resp, nil
  206. }
  207. func (ms *MasterServer) LookupEcVolume(ctx context.Context, req *master_pb.LookupEcVolumeRequest) (*master_pb.LookupEcVolumeResponse, error) {
  208. if !ms.Topo.IsLeader() {
  209. return nil, raft.NotLeaderError
  210. }
  211. resp := &master_pb.LookupEcVolumeResponse{}
  212. ecLocations, found := ms.Topo.LookupEcShards(needle.VolumeId(req.VolumeId))
  213. if !found {
  214. return resp, fmt.Errorf("ec volume %d not found", req.VolumeId)
  215. }
  216. resp.VolumeId = req.VolumeId
  217. for shardId, shardLocations := range ecLocations.Locations {
  218. var locations []*master_pb.Location
  219. for _, dn := range shardLocations {
  220. locations = append(locations, &master_pb.Location{
  221. Url: string(dn.Id()),
  222. PublicUrl: dn.PublicUrl,
  223. DataCenter: dn.GetDataCenterId(),
  224. })
  225. }
  226. resp.ShardIdLocations = append(resp.ShardIdLocations, &master_pb.LookupEcVolumeResponse_EcShardIdLocation{
  227. ShardId: uint32(shardId),
  228. Locations: locations,
  229. })
  230. }
  231. return resp, nil
  232. }
  233. func (ms *MasterServer) VacuumVolume(ctx context.Context, req *master_pb.VacuumVolumeRequest) (*master_pb.VacuumVolumeResponse, error) {
  234. if !ms.Topo.IsLeader() {
  235. return nil, raft.NotLeaderError
  236. }
  237. resp := &master_pb.VacuumVolumeResponse{}
  238. ms.Topo.Vacuum(ms.grpcDialOption, float64(req.GarbageThreshold), ms.option.MaxParallelVacuumPerServer, req.VolumeId, req.Collection, ms.preallocateSize, false)
  239. return resp, nil
  240. }
  241. func (ms *MasterServer) DisableVacuum(ctx context.Context, req *master_pb.DisableVacuumRequest) (*master_pb.DisableVacuumResponse, error) {
  242. ms.Topo.DisableVacuum()
  243. resp := &master_pb.DisableVacuumResponse{}
  244. return resp, nil
  245. }
  246. func (ms *MasterServer) EnableVacuum(ctx context.Context, req *master_pb.EnableVacuumRequest) (*master_pb.EnableVacuumResponse, error) {
  247. ms.Topo.EnableVacuum()
  248. resp := &master_pb.EnableVacuumResponse{}
  249. return resp, nil
  250. }
  251. func (ms *MasterServer) VolumeMarkReadonly(ctx context.Context, req *master_pb.VolumeMarkReadonlyRequest) (*master_pb.VolumeMarkReadonlyResponse, error) {
  252. if !ms.Topo.IsLeader() {
  253. return nil, raft.NotLeaderError
  254. }
  255. resp := &master_pb.VolumeMarkReadonlyResponse{}
  256. replicaPlacement, _ := super_block.NewReplicaPlacementFromByte(byte(req.ReplicaPlacement))
  257. vl := ms.Topo.GetVolumeLayout(req.Collection, replicaPlacement, needle.LoadTTLFromUint32(req.Ttl), types.ToDiskType(req.DiskType))
  258. dataNodes := ms.Topo.Lookup(req.Collection, needle.VolumeId(req.VolumeId))
  259. for _, dn := range dataNodes {
  260. if dn.Ip == req.Ip && dn.Port == int(req.Port) {
  261. if req.IsReadonly {
  262. vl.SetVolumeReadOnly(dn, needle.VolumeId(req.VolumeId))
  263. } else {
  264. vl.SetVolumeWritable(dn, needle.VolumeId(req.VolumeId))
  265. }
  266. }
  267. }
  268. return resp, nil
  269. }
  270. func (ms *MasterServer) VolumeGrow(ctx context.Context, req *master_pb.VolumeGrowRequest) (*master_pb.VolumeGrowResponse, error) {
  271. if !ms.Topo.IsLeader() {
  272. return nil, raft.NotLeaderError
  273. }
  274. if req.Replication == "" {
  275. req.Replication = ms.option.DefaultReplicaPlacement
  276. }
  277. replicaPlacement, err := super_block.NewReplicaPlacementFromString(req.Replication)
  278. if err != nil {
  279. return nil, err
  280. }
  281. ttl, err := needle.ReadTTL(req.Ttl)
  282. if err != nil {
  283. return nil, err
  284. }
  285. if req.DataCenter != "" && !ms.Topo.DataCenterExists(req.DataCenter) {
  286. return nil, fmt.Errorf("data center not exists")
  287. }
  288. ver := needle.GetCurrentVersion()
  289. volumeGrowOption := topology.VolumeGrowOption{
  290. Collection: req.Collection,
  291. ReplicaPlacement: replicaPlacement,
  292. Ttl: ttl,
  293. DiskType: types.ToDiskType(req.DiskType),
  294. Preallocate: ms.preallocateSize,
  295. DataCenter: req.DataCenter,
  296. Rack: req.Rack,
  297. DataNode: req.DataNode,
  298. MemoryMapMaxSizeMb: req.MemoryMapMaxSizeMb,
  299. Version: uint32(ver),
  300. }
  301. volumeGrowRequest := topology.VolumeGrowRequest{
  302. Option: &volumeGrowOption,
  303. Count: req.WritableVolumeCount,
  304. Force: true,
  305. Reason: "grpc volume grow",
  306. }
  307. replicaCount := int64(req.WritableVolumeCount * uint32(replicaPlacement.GetCopyCount()))
  308. if ms.Topo.AvailableSpaceFor(&volumeGrowOption) < replicaCount {
  309. return nil, fmt.Errorf("only %d volumes left, not enough for %d", ms.Topo.AvailableSpaceFor(&volumeGrowOption), replicaCount)
  310. }
  311. if !ms.Topo.DataCenterExists(volumeGrowOption.DataCenter) {
  312. err = fmt.Errorf("data center %v not found in topology", volumeGrowOption.DataCenter)
  313. }
  314. ms.DoAutomaticVolumeGrow(&volumeGrowRequest)
  315. return &master_pb.VolumeGrowResponse{}, nil
  316. }