master_grpc_server_volume.go 8.2 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288
  1. package weed_server
  2. import (
  3. "context"
  4. "fmt"
  5. "math/rand/v2"
  6. "strings"
  7. "sync"
  8. "time"
  9. "github.com/seaweedfs/seaweedfs/weed/topology"
  10. "github.com/seaweedfs/raft"
  11. "github.com/seaweedfs/seaweedfs/weed/glog"
  12. "github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
  13. "github.com/seaweedfs/seaweedfs/weed/security"
  14. "github.com/seaweedfs/seaweedfs/weed/storage/needle"
  15. "github.com/seaweedfs/seaweedfs/weed/storage/super_block"
  16. "github.com/seaweedfs/seaweedfs/weed/storage/types"
  17. )
  18. func (ms *MasterServer) DoAutomaticVolumeGrow(req *topology.VolumeGrowRequest) {
  19. glog.V(1).Infoln("starting automatic volume grow")
  20. start := time.Now()
  21. newVidLocations, err := ms.vg.AutomaticGrowByType(req.Option, ms.grpcDialOption, ms.Topo, req.Count)
  22. glog.V(1).Infoln("finished automatic volume grow, cost ", time.Now().Sub(start))
  23. if err != nil {
  24. glog.V(1).Infof("automatic volume grow failed: %+v", err)
  25. return
  26. }
  27. for _, newVidLocation := range newVidLocations {
  28. ms.broadcastToClients(&master_pb.KeepConnectedResponse{VolumeLocation: newVidLocation})
  29. }
  30. }
  31. func (ms *MasterServer) ProcessGrowRequest() {
  32. go func() {
  33. for {
  34. if !ms.Topo.IsLeader() {
  35. continue
  36. }
  37. dcs := ms.Topo.ListDataCenters()
  38. for _, vlc := range ms.Topo.ListVolumeLayoutCollections() {
  39. vl := vlc.VolumeLayout
  40. if vl.HasGrowRequest() {
  41. continue
  42. }
  43. if vl.ShouldGrowVolumes(vlc.Collection) {
  44. vl.AddGrowRequest()
  45. ms.volumeGrowthRequestChan <- &topology.VolumeGrowRequest{
  46. Option: vlc.ToGrowOption(),
  47. Count: vl.GetLastGrowCount(),
  48. Reason: "collection autogrow",
  49. }
  50. } else {
  51. for _, dc := range dcs {
  52. if vl.ShouldGrowVolumesByDataNode("DataCenter", dc) {
  53. vl.AddGrowRequest()
  54. volumeGrowOption := vlc.ToGrowOption()
  55. volumeGrowOption.DataCenter = dc
  56. ms.volumeGrowthRequestChan <- &topology.VolumeGrowRequest{
  57. Option: volumeGrowOption,
  58. Count: vl.GetLastGrowCount(),
  59. Force: true,
  60. Reason: "per-dc autogrow",
  61. }
  62. }
  63. }
  64. }
  65. }
  66. time.Sleep(14*time.Minute + time.Duration(120*rand.Float32())*time.Second)
  67. }
  68. }()
  69. go func() {
  70. filter := sync.Map{}
  71. for {
  72. req, ok := <-ms.volumeGrowthRequestChan
  73. if !ok {
  74. break
  75. }
  76. option := req.Option
  77. vl := ms.Topo.GetVolumeLayout(option.Collection, option.ReplicaPlacement, option.Ttl, option.DiskType)
  78. if !ms.Topo.IsLeader() {
  79. //discard buffered requests
  80. time.Sleep(time.Second * 1)
  81. vl.DoneGrowRequest()
  82. continue
  83. }
  84. // filter out identical requests being processed
  85. found := false
  86. filter.Range(func(k, v interface{}) bool {
  87. existingReq := k.(*topology.VolumeGrowRequest)
  88. if existingReq.Equals(req) {
  89. found = true
  90. }
  91. return !found
  92. })
  93. // not atomic but it's okay
  94. if found || (!req.Force && !vl.ShouldGrowVolumes(req.Option.Collection)) {
  95. glog.V(4).Infoln("discard volume grow request")
  96. time.Sleep(time.Millisecond * 211)
  97. vl.DoneGrowRequest()
  98. continue
  99. }
  100. filter.Store(req, nil)
  101. // we have lock called inside vg
  102. glog.V(0).Infof("volume grow %+v", req)
  103. go func(req *topology.VolumeGrowRequest, vl *topology.VolumeLayout) {
  104. ms.DoAutomaticVolumeGrow(req)
  105. vl.DoneGrowRequest()
  106. filter.Delete(req)
  107. }(req, vl)
  108. }
  109. }()
  110. }
  111. func (ms *MasterServer) LookupVolume(ctx context.Context, req *master_pb.LookupVolumeRequest) (*master_pb.LookupVolumeResponse, error) {
  112. resp := &master_pb.LookupVolumeResponse{}
  113. volumeLocations := ms.lookupVolumeId(req.VolumeOrFileIds, req.Collection)
  114. for _, volumeOrFileId := range req.VolumeOrFileIds {
  115. vid := volumeOrFileId
  116. commaSep := strings.Index(vid, ",")
  117. if commaSep > 0 {
  118. vid = vid[0:commaSep]
  119. }
  120. if result, found := volumeLocations[vid]; found {
  121. var locations []*master_pb.Location
  122. for _, loc := range result.Locations {
  123. locations = append(locations, &master_pb.Location{
  124. Url: loc.Url,
  125. PublicUrl: loc.PublicUrl,
  126. DataCenter: loc.DataCenter,
  127. GrpcPort: uint32(loc.GrpcPort),
  128. })
  129. }
  130. var auth string
  131. if commaSep > 0 { // this is a file id
  132. auth = string(security.GenJwtForVolumeServer(ms.guard.SigningKey, ms.guard.ExpiresAfterSec, result.VolumeOrFileId))
  133. }
  134. resp.VolumeIdLocations = append(resp.VolumeIdLocations, &master_pb.LookupVolumeResponse_VolumeIdLocation{
  135. VolumeOrFileId: result.VolumeOrFileId,
  136. Locations: locations,
  137. Error: result.Error,
  138. Auth: auth,
  139. })
  140. }
  141. }
  142. return resp, nil
  143. }
  144. func (ms *MasterServer) Statistics(ctx context.Context, req *master_pb.StatisticsRequest) (*master_pb.StatisticsResponse, error) {
  145. if !ms.Topo.IsLeader() {
  146. return nil, raft.NotLeaderError
  147. }
  148. if req.Replication == "" {
  149. req.Replication = ms.option.DefaultReplicaPlacement
  150. }
  151. replicaPlacement, err := super_block.NewReplicaPlacementFromString(req.Replication)
  152. if err != nil {
  153. return nil, err
  154. }
  155. ttl, err := needle.ReadTTL(req.Ttl)
  156. if err != nil {
  157. return nil, err
  158. }
  159. volumeLayout := ms.Topo.GetVolumeLayout(req.Collection, replicaPlacement, ttl, types.ToDiskType(req.DiskType))
  160. stats := volumeLayout.Stats()
  161. totalSize := ms.Topo.GetDiskUsages().GetMaxVolumeCount() * int64(ms.option.VolumeSizeLimitMB) * 1024 * 1024
  162. resp := &master_pb.StatisticsResponse{
  163. TotalSize: uint64(totalSize),
  164. UsedSize: stats.UsedSize,
  165. FileCount: stats.FileCount,
  166. }
  167. return resp, nil
  168. }
  169. func (ms *MasterServer) VolumeList(ctx context.Context, req *master_pb.VolumeListRequest) (*master_pb.VolumeListResponse, error) {
  170. if !ms.Topo.IsLeader() {
  171. return nil, raft.NotLeaderError
  172. }
  173. resp := &master_pb.VolumeListResponse{
  174. TopologyInfo: ms.Topo.ToTopologyInfo(),
  175. VolumeSizeLimitMb: uint64(ms.option.VolumeSizeLimitMB),
  176. }
  177. return resp, nil
  178. }
  179. func (ms *MasterServer) LookupEcVolume(ctx context.Context, req *master_pb.LookupEcVolumeRequest) (*master_pb.LookupEcVolumeResponse, error) {
  180. if !ms.Topo.IsLeader() {
  181. return nil, raft.NotLeaderError
  182. }
  183. resp := &master_pb.LookupEcVolumeResponse{}
  184. ecLocations, found := ms.Topo.LookupEcShards(needle.VolumeId(req.VolumeId))
  185. if !found {
  186. return resp, fmt.Errorf("ec volume %d not found", req.VolumeId)
  187. }
  188. resp.VolumeId = req.VolumeId
  189. for shardId, shardLocations := range ecLocations.Locations {
  190. var locations []*master_pb.Location
  191. for _, dn := range shardLocations {
  192. locations = append(locations, &master_pb.Location{
  193. Url: string(dn.Id()),
  194. PublicUrl: dn.PublicUrl,
  195. DataCenter: dn.GetDataCenterId(),
  196. })
  197. }
  198. resp.ShardIdLocations = append(resp.ShardIdLocations, &master_pb.LookupEcVolumeResponse_EcShardIdLocation{
  199. ShardId: uint32(shardId),
  200. Locations: locations,
  201. })
  202. }
  203. return resp, nil
  204. }
  205. func (ms *MasterServer) VacuumVolume(ctx context.Context, req *master_pb.VacuumVolumeRequest) (*master_pb.VacuumVolumeResponse, error) {
  206. if !ms.Topo.IsLeader() {
  207. return nil, raft.NotLeaderError
  208. }
  209. resp := &master_pb.VacuumVolumeResponse{}
  210. ms.Topo.Vacuum(ms.grpcDialOption, float64(req.GarbageThreshold), ms.option.MaxParallelVacuumPerServer, req.VolumeId, req.Collection, ms.preallocateSize)
  211. return resp, nil
  212. }
  213. func (ms *MasterServer) DisableVacuum(ctx context.Context, req *master_pb.DisableVacuumRequest) (*master_pb.DisableVacuumResponse, error) {
  214. ms.Topo.DisableVacuum()
  215. resp := &master_pb.DisableVacuumResponse{}
  216. return resp, nil
  217. }
  218. func (ms *MasterServer) EnableVacuum(ctx context.Context, req *master_pb.EnableVacuumRequest) (*master_pb.EnableVacuumResponse, error) {
  219. ms.Topo.EnableVacuum()
  220. resp := &master_pb.EnableVacuumResponse{}
  221. return resp, nil
  222. }
  223. func (ms *MasterServer) VolumeMarkReadonly(ctx context.Context, req *master_pb.VolumeMarkReadonlyRequest) (*master_pb.VolumeMarkReadonlyResponse, error) {
  224. if !ms.Topo.IsLeader() {
  225. return nil, raft.NotLeaderError
  226. }
  227. resp := &master_pb.VolumeMarkReadonlyResponse{}
  228. replicaPlacement, _ := super_block.NewReplicaPlacementFromByte(byte(req.ReplicaPlacement))
  229. vl := ms.Topo.GetVolumeLayout(req.Collection, replicaPlacement, needle.LoadTTLFromUint32(req.Ttl), types.ToDiskType(req.DiskType))
  230. dataNodes := ms.Topo.Lookup(req.Collection, needle.VolumeId(req.VolumeId))
  231. for _, dn := range dataNodes {
  232. if dn.Ip == req.Ip && dn.Port == int(req.Port) {
  233. if req.IsReadonly {
  234. vl.SetVolumeReadOnly(dn, needle.VolumeId(req.VolumeId))
  235. } else {
  236. vl.SetVolumeWritable(dn, needle.VolumeId(req.VolumeId))
  237. }
  238. }
  239. }
  240. return resp, nil
  241. }