master_grpc_server_volume.go 11 KB

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