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.

1101 lines
34 KiB

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