You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

367 lines
11 KiB

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