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.

1090 lines
33 KiB

2 months ago
4 years ago
4 years ago
  1. package shell
  2. import (
  3. "context"
  4. "errors"
  5. "fmt"
  6. "math/rand/v2"
  7. "sort"
  8. "sync"
  9. "time"
  10. "github.com/seaweedfs/seaweedfs/weed/glog"
  11. "github.com/seaweedfs/seaweedfs/weed/operation"
  12. "github.com/seaweedfs/seaweedfs/weed/pb"
  13. "github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
  14. "github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb"
  15. "github.com/seaweedfs/seaweedfs/weed/storage/erasure_coding"
  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. "golang.org/x/exp/slices"
  20. "google.golang.org/grpc"
  21. )
  22. type DataCenterId string
  23. type EcNodeId string
  24. type RackId string
  25. type EcNode struct {
  26. info *master_pb.DataNodeInfo
  27. dc DataCenterId
  28. rack RackId
  29. freeEcSlot int
  30. }
  31. type CandidateEcNode struct {
  32. ecNode *EcNode
  33. shardCount int
  34. }
  35. type EcRack struct {
  36. ecNodes map[EcNodeId]*EcNode
  37. freeEcSlot int
  38. }
  39. var (
  40. ecBalanceAlgorithmDescription = `
  41. func EcBalance() {
  42. for each collection:
  43. balanceEcVolumes(collectionName)
  44. for each rack:
  45. balanceEcRack(rack)
  46. }
  47. func balanceEcVolumes(collectionName){
  48. for each volume:
  49. doDeduplicateEcShards(volumeId)
  50. tracks rack~shardCount mapping
  51. for each volume:
  52. doBalanceEcShardsAcrossRacks(volumeId)
  53. for each volume:
  54. doBalanceEcShardsWithinRacks(volumeId)
  55. }
  56. // spread ec shards into more racks
  57. func doBalanceEcShardsAcrossRacks(volumeId){
  58. tracks rack~volumeIdShardCount mapping
  59. averageShardsPerEcRack = totalShardNumber / numRacks // totalShardNumber is 14 for now, later could varies for each dc
  60. ecShardsToMove = select overflown ec shards from racks with ec shard counts > averageShardsPerEcRack
  61. for each ecShardsToMove {
  62. destRack = pickOneRack(rack~shardCount, rack~volumeIdShardCount, ecShardReplicaPlacement)
  63. destVolumeServers = volume servers on the destRack
  64. pickOneEcNodeAndMoveOneShard(destVolumeServers)
  65. }
  66. }
  67. func doBalanceEcShardsWithinRacks(volumeId){
  68. racks = collect all racks that the volume id is on
  69. for rack, shards := range racks
  70. doBalanceEcShardsWithinOneRack(volumeId, shards, rack)
  71. }
  72. // move ec shards
  73. func doBalanceEcShardsWithinOneRack(volumeId, shards, rackId){
  74. tracks volumeServer~volumeIdShardCount mapping
  75. averageShardCount = len(shards) / numVolumeServers
  76. volumeServersOverAverage = volume servers with volumeId's ec shard counts > averageShardsPerEcRack
  77. ecShardsToMove = select overflown ec shards from volumeServersOverAverage
  78. for each ecShardsToMove {
  79. destVolumeServer = pickOneVolumeServer(volumeServer~shardCount, volumeServer~volumeIdShardCount, ecShardReplicaPlacement)
  80. pickOneEcNodeAndMoveOneShard(destVolumeServers)
  81. }
  82. }
  83. // move ec shards while keeping shard distribution for the same volume unchanged or more even
  84. func balanceEcRack(rack){
  85. averageShardCount = total shards / numVolumeServers
  86. for hasMovedOneEcShard {
  87. sort all volume servers ordered by the number of local ec shards
  88. pick the volume server A with the lowest number of ec shards x
  89. pick the volume server B with the highest number of ec shards y
  90. if y > averageShardCount and x +1 <= averageShardCount {
  91. if B has a ec shard with volume id v that A does not have {
  92. move one ec shard v from B to A
  93. hasMovedOneEcShard = true
  94. }
  95. }
  96. }
  97. }
  98. `
  99. // Overridable functions for testing.
  100. getDefaultReplicaPlacement = _getDefaultReplicaPlacement
  101. )
  102. type ErrorWaitGroup struct {
  103. parallelize bool
  104. wg *sync.WaitGroup
  105. errors []error
  106. errorsMu sync.Mutex
  107. }
  108. type ErrorWaitGroupTask func() error
  109. func (ewg *ErrorWaitGroup) Init() {
  110. if ewg.wg != nil {
  111. return
  112. }
  113. ewg.wg = &sync.WaitGroup{}
  114. ewg.errors = nil
  115. }
  116. func (ewg *ErrorWaitGroup) Add(f ErrorWaitGroupTask) {
  117. if ewg.wg == nil || !ewg.parallelize {
  118. ewg.errors = append(ewg.errors, f())
  119. return
  120. }
  121. ewg.wg.Add(1)
  122. go func() {
  123. err := f()
  124. ewg.errorsMu.Lock()
  125. ewg.errors = append(ewg.errors, err)
  126. ewg.errorsMu.Unlock()
  127. ewg.wg.Done()
  128. }()
  129. }
  130. func (ewg *ErrorWaitGroup) Wait() error {
  131. if ewg.wg != nil {
  132. ewg.wg.Wait()
  133. }
  134. err := errors.Join(ewg.errors...)
  135. ewg.wg = nil
  136. ewg.errors = nil
  137. return err
  138. }
  139. func _getDefaultReplicaPlacement(commandEnv *CommandEnv) (*super_block.ReplicaPlacement, error) {
  140. var resp *master_pb.GetMasterConfigurationResponse
  141. var err error
  142. err = commandEnv.MasterClient.WithClient(false, func(client master_pb.SeaweedClient) error {
  143. resp, err = client.GetMasterConfiguration(context.Background(), &master_pb.GetMasterConfigurationRequest{})
  144. return err
  145. })
  146. if err != nil {
  147. return nil, err
  148. }
  149. return super_block.NewReplicaPlacementFromString(resp.DefaultReplication)
  150. }
  151. func parseReplicaPlacementArg(commandEnv *CommandEnv, replicaStr string) (*super_block.ReplicaPlacement, error) {
  152. if replicaStr != "" {
  153. rp, err := super_block.NewReplicaPlacementFromString(replicaStr)
  154. if err == nil {
  155. fmt.Printf("using replica placement %q for EC volumes\n", rp.String())
  156. }
  157. return rp, err
  158. }
  159. // No replica placement argument provided, resolve from master default settings.
  160. rp, err := getDefaultReplicaPlacement(commandEnv)
  161. if err == nil {
  162. fmt.Printf("using master default replica placement %q for EC volumes\n", rp.String())
  163. }
  164. return rp, err
  165. }
  166. func collectTopologyInfo(commandEnv *CommandEnv, delayBeforeCollecting time.Duration) (topoInfo *master_pb.TopologyInfo, volumeSizeLimitMb uint64, err error) {
  167. if delayBeforeCollecting > 0 {
  168. time.Sleep(delayBeforeCollecting)
  169. }
  170. var resp *master_pb.VolumeListResponse
  171. err = commandEnv.MasterClient.WithClient(false, func(client master_pb.SeaweedClient) error {
  172. resp, err = client.VolumeList(context.Background(), &master_pb.VolumeListRequest{})
  173. return err
  174. })
  175. if err != nil {
  176. return
  177. }
  178. return resp.TopologyInfo, resp.VolumeSizeLimitMb, nil
  179. }
  180. func collectEcNodesForDC(commandEnv *CommandEnv, selectedDataCenter string) (ecNodes []*EcNode, totalFreeEcSlots int, err error) {
  181. // list all possible locations
  182. // collect topology information
  183. topologyInfo, _, err := collectTopologyInfo(commandEnv, 0)
  184. if err != nil {
  185. return
  186. }
  187. // find out all volume servers with one slot left.
  188. ecNodes, totalFreeEcSlots = collectEcVolumeServersByDc(topologyInfo, selectedDataCenter)
  189. sortEcNodesByFreeslotsDescending(ecNodes)
  190. return
  191. }
  192. func collectEcNodes(commandEnv *CommandEnv) (ecNodes []*EcNode, totalFreeEcSlots int, err error) {
  193. return collectEcNodesForDC(commandEnv, "")
  194. }
  195. func collectCollectionsForVolumeIds(t *master_pb.TopologyInfo, vids []needle.VolumeId) []string {
  196. if len(vids) == 0 {
  197. return nil
  198. }
  199. found := map[string]bool{}
  200. for _, dc := range t.DataCenterInfos {
  201. for _, r := range dc.RackInfos {
  202. for _, dn := range r.DataNodeInfos {
  203. for _, diskInfo := range dn.DiskInfos {
  204. for _, vi := range diskInfo.VolumeInfos {
  205. for _, vid := range vids {
  206. if needle.VolumeId(vi.Id) == vid && vi.Collection != "" {
  207. found[vi.Collection] = true
  208. }
  209. }
  210. }
  211. for _, ecs := range diskInfo.EcShardInfos {
  212. for _, vid := range vids {
  213. if needle.VolumeId(ecs.Id) == vid && ecs.Collection != "" {
  214. found[ecs.Collection] = true
  215. }
  216. }
  217. }
  218. }
  219. }
  220. }
  221. }
  222. if len(found) == 0 {
  223. return nil
  224. }
  225. collections := []string{}
  226. for k, _ := range found {
  227. collections = append(collections, k)
  228. }
  229. sort.Strings(collections)
  230. return collections
  231. }
  232. func moveMountedShardToEcNode(commandEnv *CommandEnv, existingLocation *EcNode, collection string, vid needle.VolumeId, shardId erasure_coding.ShardId, destinationEcNode *EcNode, applyBalancing bool) (err error) {
  233. if !commandEnv.isLocked() {
  234. return fmt.Errorf("lock is lost")
  235. }
  236. copiedShardIds := []uint32{uint32(shardId)}
  237. if applyBalancing {
  238. existingServerAddress := pb.NewServerAddressFromDataNode(existingLocation.info)
  239. // ask destination node to copy shard and the ecx file from source node, and mount it
  240. copiedShardIds, err = oneServerCopyAndMountEcShardsFromSource(commandEnv.option.GrpcDialOption, destinationEcNode, []uint32{uint32(shardId)}, vid, collection, existingServerAddress)
  241. if err != nil {
  242. return err
  243. }
  244. // unmount the to be deleted shards
  245. err = unmountEcShards(commandEnv.option.GrpcDialOption, vid, existingServerAddress, copiedShardIds)
  246. if err != nil {
  247. return err
  248. }
  249. // ask source node to delete the shard, and maybe the ecx file
  250. err = sourceServerDeleteEcShards(commandEnv.option.GrpcDialOption, collection, vid, existingServerAddress, copiedShardIds)
  251. if err != nil {
  252. return err
  253. }
  254. fmt.Printf("moved ec shard %d.%d %s => %s\n", vid, shardId, existingLocation.info.Id, destinationEcNode.info.Id)
  255. }
  256. destinationEcNode.addEcVolumeShards(vid, collection, copiedShardIds)
  257. existingLocation.deleteEcVolumeShards(vid, copiedShardIds)
  258. return nil
  259. }
  260. func oneServerCopyAndMountEcShardsFromSource(grpcDialOption grpc.DialOption,
  261. targetServer *EcNode, shardIdsToCopy []uint32,
  262. volumeId needle.VolumeId, collection string, existingLocation pb.ServerAddress) (copiedShardIds []uint32, err error) {
  263. fmt.Printf("allocate %d.%v %s => %s\n", volumeId, shardIdsToCopy, existingLocation, targetServer.info.Id)
  264. targetAddress := pb.NewServerAddressFromDataNode(targetServer.info)
  265. err = operation.WithVolumeServerClient(false, targetAddress, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
  266. if targetAddress != existingLocation {
  267. fmt.Printf("copy %d.%v %s => %s\n", volumeId, shardIdsToCopy, existingLocation, targetServer.info.Id)
  268. _, copyErr := volumeServerClient.VolumeEcShardsCopy(context.Background(), &volume_server_pb.VolumeEcShardsCopyRequest{
  269. VolumeId: uint32(volumeId),
  270. Collection: collection,
  271. ShardIds: shardIdsToCopy,
  272. CopyEcxFile: true,
  273. CopyEcjFile: true,
  274. CopyVifFile: true,
  275. SourceDataNode: string(existingLocation),
  276. })
  277. if copyErr != nil {
  278. return fmt.Errorf("copy %d.%v %s => %s : %v\n", volumeId, shardIdsToCopy, existingLocation, targetServer.info.Id, copyErr)
  279. }
  280. }
  281. fmt.Printf("mount %d.%v on %s\n", volumeId, shardIdsToCopy, targetServer.info.Id)
  282. _, mountErr := volumeServerClient.VolumeEcShardsMount(context.Background(), &volume_server_pb.VolumeEcShardsMountRequest{
  283. VolumeId: uint32(volumeId),
  284. Collection: collection,
  285. ShardIds: shardIdsToCopy,
  286. })
  287. if mountErr != nil {
  288. return fmt.Errorf("mount %d.%v on %s : %v\n", volumeId, shardIdsToCopy, targetServer.info.Id, mountErr)
  289. }
  290. if targetAddress != existingLocation {
  291. copiedShardIds = shardIdsToCopy
  292. glog.V(0).Infof("%s ec volume %d deletes shards %+v", existingLocation, volumeId, copiedShardIds)
  293. }
  294. return nil
  295. })
  296. if err != nil {
  297. return
  298. }
  299. return
  300. }
  301. func eachDataNode(topo *master_pb.TopologyInfo, fn func(dc DataCenterId, rack RackId, dn *master_pb.DataNodeInfo)) {
  302. for _, dc := range topo.DataCenterInfos {
  303. for _, rack := range dc.RackInfos {
  304. for _, dn := range rack.DataNodeInfos {
  305. fn(DataCenterId(dc.Id), RackId(rack.Id), dn)
  306. }
  307. }
  308. }
  309. }
  310. func sortEcNodesByFreeslotsDescending(ecNodes []*EcNode) {
  311. slices.SortFunc(ecNodes, func(a, b *EcNode) int {
  312. return b.freeEcSlot - a.freeEcSlot
  313. })
  314. }
  315. func sortEcNodesByFreeslotsAscending(ecNodes []*EcNode) {
  316. slices.SortFunc(ecNodes, func(a, b *EcNode) int {
  317. return a.freeEcSlot - b.freeEcSlot
  318. })
  319. }
  320. // if the index node changed the freeEcSlot, need to keep every EcNode still sorted
  321. func ensureSortedEcNodes(data []*CandidateEcNode, index int, lessThan func(i, j int) bool) {
  322. for i := index - 1; i >= 0; i-- {
  323. if lessThan(i+1, i) {
  324. swap(data, i, i+1)
  325. } else {
  326. break
  327. }
  328. }
  329. for i := index + 1; i < len(data); i++ {
  330. if lessThan(i, i-1) {
  331. swap(data, i, i-1)
  332. } else {
  333. break
  334. }
  335. }
  336. }
  337. func swap(data []*CandidateEcNode, i, j int) {
  338. t := data[i]
  339. data[i] = data[j]
  340. data[j] = t
  341. }
  342. func countShards(ecShardInfos []*master_pb.VolumeEcShardInformationMessage) (count int) {
  343. for _, ecShardInfo := range ecShardInfos {
  344. shardBits := erasure_coding.ShardBits(ecShardInfo.EcIndexBits)
  345. count += shardBits.ShardIdCount()
  346. }
  347. return
  348. }
  349. func countFreeShardSlots(dn *master_pb.DataNodeInfo, diskType types.DiskType) (count int) {
  350. if dn.DiskInfos == nil {
  351. return 0
  352. }
  353. diskInfo := dn.DiskInfos[string(diskType)]
  354. if diskInfo == nil {
  355. return 0
  356. }
  357. return int(diskInfo.MaxVolumeCount-diskInfo.VolumeCount)*erasure_coding.DataShardsCount - countShards(diskInfo.EcShardInfos)
  358. }
  359. func (ecNode *EcNode) localShardIdCount(vid uint32) int {
  360. for _, diskInfo := range ecNode.info.DiskInfos {
  361. for _, ecShardInfo := range diskInfo.EcShardInfos {
  362. if vid == ecShardInfo.Id {
  363. shardBits := erasure_coding.ShardBits(ecShardInfo.EcIndexBits)
  364. return shardBits.ShardIdCount()
  365. }
  366. }
  367. }
  368. return 0
  369. }
  370. func collectEcVolumeServersByDc(topo *master_pb.TopologyInfo, selectedDataCenter string) (ecNodes []*EcNode, totalFreeEcSlots int) {
  371. eachDataNode(topo, func(dc DataCenterId, rack RackId, dn *master_pb.DataNodeInfo) {
  372. if selectedDataCenter != "" && selectedDataCenter != string(dc) {
  373. return
  374. }
  375. freeEcSlots := countFreeShardSlots(dn, types.HardDriveType)
  376. ecNodes = append(ecNodes, &EcNode{
  377. info: dn,
  378. dc: dc,
  379. rack: rack,
  380. freeEcSlot: int(freeEcSlots),
  381. })
  382. totalFreeEcSlots += freeEcSlots
  383. })
  384. return
  385. }
  386. func sourceServerDeleteEcShards(grpcDialOption grpc.DialOption, collection string, volumeId needle.VolumeId, sourceLocation pb.ServerAddress, toBeDeletedShardIds []uint32) error {
  387. fmt.Printf("delete %d.%v from %s\n", volumeId, toBeDeletedShardIds, sourceLocation)
  388. return operation.WithVolumeServerClient(false, sourceLocation, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
  389. _, deleteErr := volumeServerClient.VolumeEcShardsDelete(context.Background(), &volume_server_pb.VolumeEcShardsDeleteRequest{
  390. VolumeId: uint32(volumeId),
  391. Collection: collection,
  392. ShardIds: toBeDeletedShardIds,
  393. })
  394. return deleteErr
  395. })
  396. }
  397. func unmountEcShards(grpcDialOption grpc.DialOption, volumeId needle.VolumeId, sourceLocation pb.ServerAddress, toBeUnmountedhardIds []uint32) error {
  398. fmt.Printf("unmount %d.%v from %s\n", volumeId, toBeUnmountedhardIds, sourceLocation)
  399. return operation.WithVolumeServerClient(false, sourceLocation, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
  400. _, deleteErr := volumeServerClient.VolumeEcShardsUnmount(context.Background(), &volume_server_pb.VolumeEcShardsUnmountRequest{
  401. VolumeId: uint32(volumeId),
  402. ShardIds: toBeUnmountedhardIds,
  403. })
  404. return deleteErr
  405. })
  406. }
  407. func mountEcShards(grpcDialOption grpc.DialOption, collection string, volumeId needle.VolumeId, sourceLocation pb.ServerAddress, toBeMountedhardIds []uint32) error {
  408. fmt.Printf("mount %d.%v on %s\n", volumeId, toBeMountedhardIds, sourceLocation)
  409. return operation.WithVolumeServerClient(false, sourceLocation, grpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
  410. _, mountErr := volumeServerClient.VolumeEcShardsMount(context.Background(), &volume_server_pb.VolumeEcShardsMountRequest{
  411. VolumeId: uint32(volumeId),
  412. Collection: collection,
  413. ShardIds: toBeMountedhardIds,
  414. })
  415. return mountErr
  416. })
  417. }
  418. func ceilDivide(a, b int) int {
  419. var r int
  420. if (a % b) != 0 {
  421. r = 1
  422. }
  423. return (a / b) + r
  424. }
  425. func findEcVolumeShards(ecNode *EcNode, vid needle.VolumeId) erasure_coding.ShardBits {
  426. if diskInfo, found := ecNode.info.DiskInfos[string(types.HardDriveType)]; found {
  427. for _, shardInfo := range diskInfo.EcShardInfos {
  428. if needle.VolumeId(shardInfo.Id) == vid {
  429. return erasure_coding.ShardBits(shardInfo.EcIndexBits)
  430. }
  431. }
  432. }
  433. return 0
  434. }
  435. func (ecNode *EcNode) addEcVolumeShards(vid needle.VolumeId, collection string, shardIds []uint32) *EcNode {
  436. foundVolume := false
  437. diskInfo, found := ecNode.info.DiskInfos[string(types.HardDriveType)]
  438. if found {
  439. for _, shardInfo := range diskInfo.EcShardInfos {
  440. if needle.VolumeId(shardInfo.Id) == vid {
  441. oldShardBits := erasure_coding.ShardBits(shardInfo.EcIndexBits)
  442. newShardBits := oldShardBits
  443. for _, shardId := range shardIds {
  444. newShardBits = newShardBits.AddShardId(erasure_coding.ShardId(shardId))
  445. }
  446. shardInfo.EcIndexBits = uint32(newShardBits)
  447. ecNode.freeEcSlot -= newShardBits.ShardIdCount() - oldShardBits.ShardIdCount()
  448. foundVolume = true
  449. break
  450. }
  451. }
  452. } else {
  453. diskInfo = &master_pb.DiskInfo{
  454. Type: string(types.HardDriveType),
  455. }
  456. ecNode.info.DiskInfos[string(types.HardDriveType)] = diskInfo
  457. }
  458. if !foundVolume {
  459. var newShardBits erasure_coding.ShardBits
  460. for _, shardId := range shardIds {
  461. newShardBits = newShardBits.AddShardId(erasure_coding.ShardId(shardId))
  462. }
  463. diskInfo.EcShardInfos = append(diskInfo.EcShardInfos, &master_pb.VolumeEcShardInformationMessage{
  464. Id: uint32(vid),
  465. Collection: collection,
  466. EcIndexBits: uint32(newShardBits),
  467. DiskType: string(types.HardDriveType),
  468. })
  469. ecNode.freeEcSlot -= len(shardIds)
  470. }
  471. return ecNode
  472. }
  473. func (ecNode *EcNode) deleteEcVolumeShards(vid needle.VolumeId, shardIds []uint32) *EcNode {
  474. if diskInfo, found := ecNode.info.DiskInfos[string(types.HardDriveType)]; found {
  475. for _, shardInfo := range diskInfo.EcShardInfos {
  476. if needle.VolumeId(shardInfo.Id) == vid {
  477. oldShardBits := erasure_coding.ShardBits(shardInfo.EcIndexBits)
  478. newShardBits := oldShardBits
  479. for _, shardId := range shardIds {
  480. newShardBits = newShardBits.RemoveShardId(erasure_coding.ShardId(shardId))
  481. }
  482. shardInfo.EcIndexBits = uint32(newShardBits)
  483. ecNode.freeEcSlot -= newShardBits.ShardIdCount() - oldShardBits.ShardIdCount()
  484. }
  485. }
  486. }
  487. return ecNode
  488. }
  489. func groupByCount(data []*EcNode, identifierFn func(*EcNode) (id string, count int)) map[string]int {
  490. countMap := make(map[string]int)
  491. for _, d := range data {
  492. id, count := identifierFn(d)
  493. countMap[id] += count
  494. }
  495. return countMap
  496. }
  497. func groupBy(data []*EcNode, identifierFn func(*EcNode) (id string)) map[string][]*EcNode {
  498. groupMap := make(map[string][]*EcNode)
  499. for _, d := range data {
  500. id := identifierFn(d)
  501. groupMap[id] = append(groupMap[id], d)
  502. }
  503. return groupMap
  504. }
  505. type ecBalancer struct {
  506. commandEnv *CommandEnv
  507. ecNodes []*EcNode
  508. replicaPlacement *super_block.ReplicaPlacement
  509. applyBalancing bool
  510. ewg ErrorWaitGroup
  511. }
  512. func (ecb *ecBalancer) racks() map[RackId]*EcRack {
  513. racks := make(map[RackId]*EcRack)
  514. for _, ecNode := range ecb.ecNodes {
  515. if racks[ecNode.rack] == nil {
  516. racks[ecNode.rack] = &EcRack{
  517. ecNodes: make(map[EcNodeId]*EcNode),
  518. }
  519. }
  520. racks[ecNode.rack].ecNodes[EcNodeId(ecNode.info.Id)] = ecNode
  521. racks[ecNode.rack].freeEcSlot += ecNode.freeEcSlot
  522. }
  523. return racks
  524. }
  525. func (ecb *ecBalancer) balanceEcVolumes(collection string) error {
  526. fmt.Printf("balanceEcVolumes %s\n", collection)
  527. if err := ecb.deleteDuplicatedEcShards(collection); err != nil {
  528. return fmt.Errorf("delete duplicated collection %s ec shards: %v", collection, err)
  529. }
  530. if err := ecb.balanceEcShardsAcrossRacks(collection); err != nil {
  531. return fmt.Errorf("balance across racks collection %s ec shards: %v", collection, err)
  532. }
  533. if err := ecb.balanceEcShardsWithinRacks(collection); err != nil {
  534. return fmt.Errorf("balance within racks collection %s ec shards: %v", collection, err)
  535. }
  536. return nil
  537. }
  538. func (ecb *ecBalancer) deleteDuplicatedEcShards(collection string) error {
  539. vidLocations := ecb.collectVolumeIdToEcNodes(collection)
  540. ecb.ewg.Init()
  541. for vid, locations := range vidLocations {
  542. ecb.ewg.Add(func() error {
  543. return ecb.doDeduplicateEcShards(collection, vid, locations)
  544. })
  545. }
  546. return ecb.ewg.Wait()
  547. }
  548. func (ecb *ecBalancer) doDeduplicateEcShards(collection string, vid needle.VolumeId, locations []*EcNode) error {
  549. // check whether this volume has ecNodes that are over average
  550. shardToLocations := make([][]*EcNode, erasure_coding.TotalShardsCount)
  551. for _, ecNode := range locations {
  552. shardBits := findEcVolumeShards(ecNode, vid)
  553. for _, shardId := range shardBits.ShardIds() {
  554. shardToLocations[shardId] = append(shardToLocations[shardId], ecNode)
  555. }
  556. }
  557. for shardId, ecNodes := range shardToLocations {
  558. if len(ecNodes) <= 1 {
  559. continue
  560. }
  561. sortEcNodesByFreeslotsAscending(ecNodes)
  562. fmt.Printf("ec shard %d.%d has %d copies, keeping %v\n", vid, shardId, len(ecNodes), ecNodes[0].info.Id)
  563. if !ecb.applyBalancing {
  564. continue
  565. }
  566. duplicatedShardIds := []uint32{uint32(shardId)}
  567. for _, ecNode := range ecNodes[1:] {
  568. if err := unmountEcShards(ecb.commandEnv.option.GrpcDialOption, vid, pb.NewServerAddressFromDataNode(ecNode.info), duplicatedShardIds); err != nil {
  569. return err
  570. }
  571. if err := sourceServerDeleteEcShards(ecb.commandEnv.option.GrpcDialOption, collection, vid, pb.NewServerAddressFromDataNode(ecNode.info), duplicatedShardIds); err != nil {
  572. return err
  573. }
  574. ecNode.deleteEcVolumeShards(vid, duplicatedShardIds)
  575. }
  576. }
  577. return nil
  578. }
  579. func (ecb *ecBalancer) balanceEcShardsAcrossRacks(collection string) error {
  580. // collect vid => []ecNode, since previous steps can change the locations
  581. vidLocations := ecb.collectVolumeIdToEcNodes(collection)
  582. // spread the ec shards evenly
  583. ecb.ewg.Init()
  584. for vid, locations := range vidLocations {
  585. ecb.ewg.Add(func() error {
  586. return ecb.doBalanceEcShardsAcrossRacks(collection, vid, locations)
  587. })
  588. }
  589. return ecb.ewg.Wait()
  590. }
  591. func countShardsByRack(vid needle.VolumeId, locations []*EcNode) map[string]int {
  592. return groupByCount(locations, func(ecNode *EcNode) (id string, count int) {
  593. shardBits := findEcVolumeShards(ecNode, vid)
  594. return string(ecNode.rack), shardBits.ShardIdCount()
  595. })
  596. }
  597. func (ecb *ecBalancer) doBalanceEcShardsAcrossRacks(collection string, vid needle.VolumeId, locations []*EcNode) error {
  598. racks := ecb.racks()
  599. // calculate average number of shards an ec rack should have for one volume
  600. averageShardsPerEcRack := ceilDivide(erasure_coding.TotalShardsCount, len(racks))
  601. // see the volume's shards are in how many racks, and how many in each rack
  602. rackToShardCount := countShardsByRack(vid, locations)
  603. rackEcNodesWithVid := groupBy(locations, func(ecNode *EcNode) string {
  604. return string(ecNode.rack)
  605. })
  606. // ecShardsToMove = select overflown ec shards from racks with ec shard counts > averageShardsPerEcRack
  607. ecShardsToMove := make(map[erasure_coding.ShardId]*EcNode)
  608. for rackId, count := range rackToShardCount {
  609. if count <= averageShardsPerEcRack {
  610. continue
  611. }
  612. possibleEcNodes := rackEcNodesWithVid[rackId]
  613. for shardId, ecNode := range pickNEcShardsToMoveFrom(possibleEcNodes, vid, count-averageShardsPerEcRack) {
  614. ecShardsToMove[shardId] = ecNode
  615. }
  616. }
  617. for shardId, ecNode := range ecShardsToMove {
  618. rackId, err := ecb.pickRackToBalanceShardsInto(racks, rackToShardCount)
  619. if err != nil {
  620. fmt.Printf("ec shard %d.%d at %s can not find a destination rack:\n%s\n", vid, shardId, ecNode.info.Id, err.Error())
  621. continue
  622. }
  623. var possibleDestinationEcNodes []*EcNode
  624. for _, n := range racks[rackId].ecNodes {
  625. possibleDestinationEcNodes = append(possibleDestinationEcNodes, n)
  626. }
  627. err = ecb.pickOneEcNodeAndMoveOneShard(ecNode, collection, vid, shardId, possibleDestinationEcNodes)
  628. if err != nil {
  629. return err
  630. }
  631. rackToShardCount[string(rackId)] += 1
  632. rackToShardCount[string(ecNode.rack)] -= 1
  633. racks[rackId].freeEcSlot -= 1
  634. racks[ecNode.rack].freeEcSlot += 1
  635. }
  636. return nil
  637. }
  638. func (ecb *ecBalancer) pickRackToBalanceShardsInto(rackToEcNodes map[RackId]*EcRack, rackToShardCount map[string]int) (RackId, error) {
  639. targets := []RackId{}
  640. targetShards := -1
  641. for _, shards := range rackToShardCount {
  642. if shards > targetShards {
  643. targetShards = shards
  644. }
  645. }
  646. details := ""
  647. for rackId, rack := range rackToEcNodes {
  648. shards := rackToShardCount[string(rackId)]
  649. if rack.freeEcSlot <= 0 {
  650. details += fmt.Sprintf(" Skipped %s because it has no free slots\n", rackId)
  651. continue
  652. }
  653. if ecb.replicaPlacement != nil && shards >= ecb.replicaPlacement.DiffRackCount {
  654. details += fmt.Sprintf(" Skipped %s because shards %d >= replica placement limit for other racks (%d)\n", rackId, shards, ecb.replicaPlacement.DiffRackCount)
  655. continue
  656. }
  657. if shards < targetShards {
  658. // Favor racks with less shards, to ensure an uniform distribution.
  659. targets = nil
  660. targetShards = shards
  661. }
  662. if shards == targetShards {
  663. targets = append(targets, rackId)
  664. }
  665. }
  666. if len(targets) == 0 {
  667. return "", errors.New(details)
  668. }
  669. return targets[rand.IntN(len(targets))], nil
  670. }
  671. func (ecb *ecBalancer) balanceEcShardsWithinRacks(collection string) error {
  672. // collect vid => []ecNode, since previous steps can change the locations
  673. vidLocations := ecb.collectVolumeIdToEcNodes(collection)
  674. racks := ecb.racks()
  675. // spread the ec shards evenly
  676. ecb.ewg.Init()
  677. for vid, locations := range vidLocations {
  678. // see the volume's shards are in how many racks, and how many in each rack
  679. rackToShardCount := countShardsByRack(vid, locations)
  680. rackEcNodesWithVid := groupBy(locations, func(ecNode *EcNode) string {
  681. return string(ecNode.rack)
  682. })
  683. for rackId, _ := range rackToShardCount {
  684. var possibleDestinationEcNodes []*EcNode
  685. for _, n := range racks[RackId(rackId)].ecNodes {
  686. if _, found := n.info.DiskInfos[string(types.HardDriveType)]; found {
  687. possibleDestinationEcNodes = append(possibleDestinationEcNodes, n)
  688. }
  689. }
  690. sourceEcNodes := rackEcNodesWithVid[rackId]
  691. averageShardsPerEcNode := ceilDivide(rackToShardCount[rackId], len(possibleDestinationEcNodes))
  692. ecb.ewg.Add(func() error {
  693. return ecb.doBalanceEcShardsWithinOneRack(averageShardsPerEcNode, collection, vid, sourceEcNodes, possibleDestinationEcNodes)
  694. })
  695. }
  696. }
  697. return ecb.ewg.Wait()
  698. }
  699. func (ecb *ecBalancer) doBalanceEcShardsWithinOneRack(averageShardsPerEcNode int, collection string, vid needle.VolumeId, existingLocations, possibleDestinationEcNodes []*EcNode) error {
  700. for _, ecNode := range existingLocations {
  701. shardBits := findEcVolumeShards(ecNode, vid)
  702. overLimitCount := shardBits.ShardIdCount() - averageShardsPerEcNode
  703. for _, shardId := range shardBits.ShardIds() {
  704. if overLimitCount <= 0 {
  705. break
  706. }
  707. fmt.Printf("%s has %d overlimit, moving ec shard %d.%d\n", ecNode.info.Id, overLimitCount, vid, shardId)
  708. err := ecb.pickOneEcNodeAndMoveOneShard(ecNode, collection, vid, shardId, possibleDestinationEcNodes)
  709. if err != nil {
  710. return err
  711. }
  712. overLimitCount--
  713. }
  714. }
  715. return nil
  716. }
  717. func (ecb *ecBalancer) balanceEcRacks() error {
  718. // balance one rack for all ec shards
  719. ecb.ewg.Init()
  720. for _, ecRack := range ecb.racks() {
  721. ecb.ewg.Add(func() error {
  722. return ecb.doBalanceEcRack(ecRack)
  723. })
  724. }
  725. return ecb.ewg.Wait()
  726. }
  727. func (ecb *ecBalancer) doBalanceEcRack(ecRack *EcRack) error {
  728. if len(ecRack.ecNodes) <= 1 {
  729. return nil
  730. }
  731. var rackEcNodes []*EcNode
  732. for _, node := range ecRack.ecNodes {
  733. rackEcNodes = append(rackEcNodes, node)
  734. }
  735. ecNodeIdToShardCount := groupByCount(rackEcNodes, func(ecNode *EcNode) (id string, count int) {
  736. diskInfo, found := ecNode.info.DiskInfos[string(types.HardDriveType)]
  737. if !found {
  738. return
  739. }
  740. for _, ecShardInfo := range diskInfo.EcShardInfos {
  741. count += erasure_coding.ShardBits(ecShardInfo.EcIndexBits).ShardIdCount()
  742. }
  743. return ecNode.info.Id, count
  744. })
  745. var totalShardCount int
  746. for _, count := range ecNodeIdToShardCount {
  747. totalShardCount += count
  748. }
  749. averageShardCount := ceilDivide(totalShardCount, len(rackEcNodes))
  750. hasMove := true
  751. for hasMove {
  752. hasMove = false
  753. slices.SortFunc(rackEcNodes, func(a, b *EcNode) int {
  754. return b.freeEcSlot - a.freeEcSlot
  755. })
  756. emptyNode, fullNode := rackEcNodes[0], rackEcNodes[len(rackEcNodes)-1]
  757. emptyNodeShardCount, fullNodeShardCount := ecNodeIdToShardCount[emptyNode.info.Id], ecNodeIdToShardCount[fullNode.info.Id]
  758. if fullNodeShardCount > averageShardCount && emptyNodeShardCount+1 <= averageShardCount {
  759. emptyNodeIds := make(map[uint32]bool)
  760. if emptyDiskInfo, found := emptyNode.info.DiskInfos[string(types.HardDriveType)]; found {
  761. for _, shards := range emptyDiskInfo.EcShardInfos {
  762. emptyNodeIds[shards.Id] = true
  763. }
  764. }
  765. if fullDiskInfo, found := fullNode.info.DiskInfos[string(types.HardDriveType)]; found {
  766. for _, shards := range fullDiskInfo.EcShardInfos {
  767. if _, found := emptyNodeIds[shards.Id]; !found {
  768. for _, shardId := range erasure_coding.ShardBits(shards.EcIndexBits).ShardIds() {
  769. fmt.Printf("%s moves ec shards %d.%d to %s\n", fullNode.info.Id, shards.Id, shardId, emptyNode.info.Id)
  770. err := moveMountedShardToEcNode(ecb.commandEnv, fullNode, shards.Collection, needle.VolumeId(shards.Id), shardId, emptyNode, ecb.applyBalancing)
  771. if err != nil {
  772. return err
  773. }
  774. ecNodeIdToShardCount[emptyNode.info.Id]++
  775. ecNodeIdToShardCount[fullNode.info.Id]--
  776. hasMove = true
  777. break
  778. }
  779. break
  780. }
  781. }
  782. }
  783. }
  784. }
  785. return nil
  786. }
  787. func (ecb *ecBalancer) pickEcNodeToBalanceShardsInto(vid needle.VolumeId, existingLocation *EcNode, possibleDestinations []*EcNode) (*EcNode, error) {
  788. if existingLocation == nil {
  789. return nil, fmt.Errorf("INTERNAL: missing source nodes")
  790. }
  791. if len(possibleDestinations) == 0 {
  792. return nil, fmt.Errorf("INTERNAL: missing destination nodes")
  793. }
  794. nodeShards := map[*EcNode]int{}
  795. for _, node := range possibleDestinations {
  796. nodeShards[node] = findEcVolumeShards(node, vid).ShardIdCount()
  797. }
  798. targets := []*EcNode{}
  799. targetShards := -1
  800. for _, shards := range nodeShards {
  801. if shards > targetShards {
  802. targetShards = shards
  803. }
  804. }
  805. details := ""
  806. for _, node := range possibleDestinations {
  807. if node.info.Id == existingLocation.info.Id {
  808. continue
  809. }
  810. if node.freeEcSlot <= 0 {
  811. details += fmt.Sprintf(" Skipped %s because it has no free slots\n", node.info.Id)
  812. continue
  813. }
  814. shards := nodeShards[node]
  815. if ecb.replicaPlacement != nil && shards >= ecb.replicaPlacement.SameRackCount {
  816. details += fmt.Sprintf(" Skipped %s because shards %d >= replica placement limit for the rack (%d)\n", node.info.Id, shards, ecb.replicaPlacement.SameRackCount)
  817. continue
  818. }
  819. if shards < targetShards {
  820. // Favor nodes with less shards, to ensure an uniform distribution.
  821. targets = nil
  822. targetShards = shards
  823. }
  824. if shards == targetShards {
  825. targets = append(targets, node)
  826. }
  827. }
  828. if len(targets) == 0 {
  829. return nil, errors.New(details)
  830. }
  831. return targets[rand.IntN(len(targets))], nil
  832. }
  833. func (ecb *ecBalancer) pickOneEcNodeAndMoveOneShard(existingLocation *EcNode, collection string, vid needle.VolumeId, shardId erasure_coding.ShardId, possibleDestinationEcNodes []*EcNode) error {
  834. destNode, err := ecb.pickEcNodeToBalanceShardsInto(vid, existingLocation, possibleDestinationEcNodes)
  835. if err != nil {
  836. fmt.Printf("WARNING: Could not find suitable taget node for %d.%d:\n%s", vid, shardId, err.Error())
  837. return nil
  838. }
  839. fmt.Printf("%s moves ec shard %d.%d to %s\n", existingLocation.info.Id, vid, shardId, destNode.info.Id)
  840. return moveMountedShardToEcNode(ecb.commandEnv, existingLocation, collection, vid, shardId, destNode, ecb.applyBalancing)
  841. }
  842. func pickNEcShardsToMoveFrom(ecNodes []*EcNode, vid needle.VolumeId, n int) map[erasure_coding.ShardId]*EcNode {
  843. picked := make(map[erasure_coding.ShardId]*EcNode)
  844. var candidateEcNodes []*CandidateEcNode
  845. for _, ecNode := range ecNodes {
  846. shardBits := findEcVolumeShards(ecNode, vid)
  847. if shardBits.ShardIdCount() > 0 {
  848. candidateEcNodes = append(candidateEcNodes, &CandidateEcNode{
  849. ecNode: ecNode,
  850. shardCount: shardBits.ShardIdCount(),
  851. })
  852. }
  853. }
  854. slices.SortFunc(candidateEcNodes, func(a, b *CandidateEcNode) int {
  855. return b.shardCount - a.shardCount
  856. })
  857. for i := 0; i < n; i++ {
  858. selectedEcNodeIndex := -1
  859. for i, candidateEcNode := range candidateEcNodes {
  860. shardBits := findEcVolumeShards(candidateEcNode.ecNode, vid)
  861. if shardBits > 0 {
  862. selectedEcNodeIndex = i
  863. for _, shardId := range shardBits.ShardIds() {
  864. candidateEcNode.shardCount--
  865. picked[shardId] = candidateEcNode.ecNode
  866. candidateEcNode.ecNode.deleteEcVolumeShards(vid, []uint32{uint32(shardId)})
  867. break
  868. }
  869. break
  870. }
  871. }
  872. if selectedEcNodeIndex >= 0 {
  873. ensureSortedEcNodes(candidateEcNodes, selectedEcNodeIndex, func(i, j int) bool {
  874. return candidateEcNodes[i].shardCount > candidateEcNodes[j].shardCount
  875. })
  876. }
  877. }
  878. return picked
  879. }
  880. func (ecb *ecBalancer) collectVolumeIdToEcNodes(collection string) map[needle.VolumeId][]*EcNode {
  881. vidLocations := make(map[needle.VolumeId][]*EcNode)
  882. for _, ecNode := range ecb.ecNodes {
  883. diskInfo, found := ecNode.info.DiskInfos[string(types.HardDriveType)]
  884. if !found {
  885. continue
  886. }
  887. for _, shardInfo := range diskInfo.EcShardInfos {
  888. // ignore if not in current collection
  889. if shardInfo.Collection == collection {
  890. vidLocations[needle.VolumeId(shardInfo.Id)] = append(vidLocations[needle.VolumeId(shardInfo.Id)], ecNode)
  891. }
  892. }
  893. }
  894. return vidLocations
  895. }
  896. func EcBalance(commandEnv *CommandEnv, collections []string, dc string, ecReplicaPlacement *super_block.ReplicaPlacement, parallelize bool, applyBalancing bool) (err error) {
  897. if len(collections) == 0 {
  898. return fmt.Errorf("no collections to balance")
  899. }
  900. // collect all ec nodes
  901. allEcNodes, totalFreeEcSlots, err := collectEcNodesForDC(commandEnv, dc)
  902. if err != nil {
  903. return err
  904. }
  905. if totalFreeEcSlots < 1 {
  906. return fmt.Errorf("no free ec shard slots. only %d left", totalFreeEcSlots)
  907. }
  908. ecb := &ecBalancer{
  909. commandEnv: commandEnv,
  910. ecNodes: allEcNodes,
  911. replicaPlacement: ecReplicaPlacement,
  912. applyBalancing: applyBalancing,
  913. ewg: ErrorWaitGroup{
  914. parallelize: parallelize,
  915. },
  916. }
  917. for _, c := range collections {
  918. if err = ecb.balanceEcVolumes(c); err != nil {
  919. return err
  920. }
  921. }
  922. if err := ecb.balanceEcRacks(); err != nil {
  923. return fmt.Errorf("balance ec racks: %v", err)
  924. }
  925. return nil
  926. }