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.

353 lines
10 KiB

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