command_ec_common.go 34 KB

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