command_volume_fix_replication.go 18 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573
  1. package shell
  2. import (
  3. "context"
  4. "flag"
  5. "fmt"
  6. "github.com/seaweedfs/seaweedfs/weed/pb"
  7. "github.com/seaweedfs/seaweedfs/weed/storage/needle"
  8. "github.com/seaweedfs/seaweedfs/weed/storage/types"
  9. "golang.org/x/exp/slices"
  10. "io"
  11. "path/filepath"
  12. "strconv"
  13. "time"
  14. "github.com/seaweedfs/seaweedfs/weed/operation"
  15. "github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
  16. "github.com/seaweedfs/seaweedfs/weed/pb/volume_server_pb"
  17. "github.com/seaweedfs/seaweedfs/weed/storage/super_block"
  18. )
  19. func init() {
  20. Commands = append(Commands, &commandVolumeFixReplication{})
  21. }
  22. type commandVolumeFixReplication struct {
  23. collectionPattern *string
  24. }
  25. func (c *commandVolumeFixReplication) Name() string {
  26. return "volume.fix.replication"
  27. }
  28. func (c *commandVolumeFixReplication) Help() string {
  29. return `add or remove replicas to volumes that are missing replicas or over-replicated
  30. This command finds all over-replicated volumes. If found, it will purge the oldest copies and stop.
  31. This command also finds all under-replicated volumes, and finds volume servers with free slots.
  32. If the free slots satisfy the replication requirement, the volume content is copied over and mounted.
  33. volume.fix.replication -n # do not take action
  34. volume.fix.replication # actually deleting or copying the volume files and mount the volume
  35. volume.fix.replication -collectionPattern=important* # fix any collections with prefix "important"
  36. Note:
  37. * each time this will only add back one replica for each volume id that is under replicated.
  38. If there are multiple replicas are missing, e.g. replica count is > 2, you may need to run this multiple times.
  39. * do not run this too quickly within seconds, since the new volume replica may take a few seconds
  40. to register itself to the master.
  41. `
  42. }
  43. func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
  44. volFixReplicationCommand := flag.NewFlagSet(c.Name(), flag.ContinueOnError)
  45. c.collectionPattern = volFixReplicationCommand.String("collectionPattern", "", "match with wildcard characters '*' and '?'")
  46. skipChange := volFixReplicationCommand.Bool("n", false, "skip the changes")
  47. noDelete := volFixReplicationCommand.Bool("noDelete", false, "Do not delete over-replicated volumes, only fix under-replication")
  48. retryCount := volFixReplicationCommand.Int("retry", 5, "how many times to retry")
  49. volumesPerStep := volFixReplicationCommand.Int("volumesPerStep", 0, "how many volumes to fix in one cycle")
  50. if err = volFixReplicationCommand.Parse(args); err != nil {
  51. return nil
  52. }
  53. if err = commandEnv.confirmIsLocked(args); err != nil {
  54. return
  55. }
  56. takeAction := !*skipChange
  57. doDeletes := !*noDelete
  58. underReplicatedVolumeIdsCount := 1
  59. for underReplicatedVolumeIdsCount > 0 {
  60. fixedVolumeReplicas := map[string]int{}
  61. // collect topology information
  62. topologyInfo, _, err := collectTopologyInfo(commandEnv, 15*time.Second)
  63. if err != nil {
  64. return err
  65. }
  66. // find all volumes that needs replication
  67. // collect all data nodes
  68. volumeReplicas, allLocations := collectVolumeReplicaLocations(topologyInfo)
  69. if len(allLocations) == 0 {
  70. return fmt.Errorf("no data nodes at all")
  71. }
  72. // find all under replicated volumes
  73. var underReplicatedVolumeIds, overReplicatedVolumeIds, misplacedVolumeIds []uint32
  74. for vid, replicas := range volumeReplicas {
  75. replica := replicas[0]
  76. replicaPlacement, _ := super_block.NewReplicaPlacementFromByte(byte(replica.info.ReplicaPlacement))
  77. if replicaPlacement.GetCopyCount() > len(replicas) {
  78. underReplicatedVolumeIds = append(underReplicatedVolumeIds, vid)
  79. } else if replicaPlacement.GetCopyCount() < len(replicas) {
  80. overReplicatedVolumeIds = append(overReplicatedVolumeIds, vid)
  81. fmt.Fprintf(writer, "volume %d replication %s, but over replicated %+d\n", replica.info.Id, replicaPlacement, len(replicas))
  82. } else if isMisplaced(replicas, replicaPlacement) {
  83. misplacedVolumeIds = append(misplacedVolumeIds, vid)
  84. fmt.Fprintf(writer, "volume %d replication %s is not well placed %+v\n", replica.info.Id, replicaPlacement, replicas)
  85. }
  86. }
  87. if !commandEnv.isLocked() {
  88. return fmt.Errorf("lock is lost")
  89. }
  90. if len(overReplicatedVolumeIds) > 0 && doDeletes {
  91. if err := c.deleteOneVolume(commandEnv, writer, takeAction, overReplicatedVolumeIds, volumeReplicas, allLocations, pickOneReplicaToDelete); err != nil {
  92. return err
  93. }
  94. }
  95. if len(misplacedVolumeIds) > 0 && doDeletes {
  96. if err := c.deleteOneVolume(commandEnv, writer, takeAction, misplacedVolumeIds, volumeReplicas, allLocations, pickOneMisplacedVolume); err != nil {
  97. return err
  98. }
  99. }
  100. underReplicatedVolumeIdsCount = len(underReplicatedVolumeIds)
  101. if underReplicatedVolumeIdsCount > 0 {
  102. // find the most under populated data nodes
  103. fixedVolumeReplicas, err = c.fixUnderReplicatedVolumes(commandEnv, writer, takeAction, underReplicatedVolumeIds, volumeReplicas, allLocations, *retryCount, *volumesPerStep)
  104. if err != nil {
  105. return err
  106. }
  107. }
  108. if *skipChange {
  109. break
  110. }
  111. // check that the topology has been updated
  112. if len(fixedVolumeReplicas) > 0 {
  113. fixedVolumes := make([]string, 0, len(fixedVolumeReplicas))
  114. for k, _ := range fixedVolumeReplicas {
  115. fixedVolumes = append(fixedVolumes, k)
  116. }
  117. volumeIdLocations, err := lookupVolumeIds(commandEnv, fixedVolumes)
  118. if err != nil {
  119. return err
  120. }
  121. for _, volumeIdLocation := range volumeIdLocations {
  122. volumeId := volumeIdLocation.VolumeOrFileId
  123. volumeIdLocationCount := len(volumeIdLocation.Locations)
  124. i := 0
  125. for fixedVolumeReplicas[volumeId] >= volumeIdLocationCount {
  126. fmt.Fprintf(writer, "the number of locations for volume %s has not increased yet, let's wait\n", volumeId)
  127. time.Sleep(time.Duration(i+1) * time.Second * 7)
  128. volumeLocIds, err := lookupVolumeIds(commandEnv, []string{volumeId})
  129. if err != nil {
  130. return err
  131. }
  132. volumeIdLocationCount = len(volumeLocIds[0].Locations)
  133. if *retryCount <= i {
  134. return fmt.Errorf("replicas volume %s mismatch in topology", volumeId)
  135. }
  136. i += 1
  137. }
  138. }
  139. }
  140. }
  141. return nil
  142. }
  143. func collectVolumeReplicaLocations(topologyInfo *master_pb.TopologyInfo) (map[uint32][]*VolumeReplica, []location) {
  144. volumeReplicas := make(map[uint32][]*VolumeReplica)
  145. var allLocations []location
  146. eachDataNode(topologyInfo, func(dc string, rack RackId, dn *master_pb.DataNodeInfo) {
  147. loc := newLocation(dc, string(rack), dn)
  148. for _, diskInfo := range dn.DiskInfos {
  149. for _, v := range diskInfo.VolumeInfos {
  150. volumeReplicas[v.Id] = append(volumeReplicas[v.Id], &VolumeReplica{
  151. location: &loc,
  152. info: v,
  153. })
  154. }
  155. }
  156. allLocations = append(allLocations, loc)
  157. })
  158. return volumeReplicas, allLocations
  159. }
  160. type SelectOneVolumeFunc func(replicas []*VolumeReplica, replicaPlacement *super_block.ReplicaPlacement) *VolumeReplica
  161. func (c *commandVolumeFixReplication) deleteOneVolume(commandEnv *CommandEnv, writer io.Writer, takeAction bool, overReplicatedVolumeIds []uint32, volumeReplicas map[uint32][]*VolumeReplica, allLocations []location, selectOneVolumeFn SelectOneVolumeFunc) error {
  162. for _, vid := range overReplicatedVolumeIds {
  163. replicas := volumeReplicas[vid]
  164. replicaPlacement, _ := super_block.NewReplicaPlacementFromByte(byte(replicas[0].info.ReplicaPlacement))
  165. replica := selectOneVolumeFn(replicas, replicaPlacement)
  166. // check collection name pattern
  167. if *c.collectionPattern != "" {
  168. matched, err := filepath.Match(*c.collectionPattern, replica.info.Collection)
  169. if err != nil {
  170. return fmt.Errorf("match pattern %s with collection %s: %v", *c.collectionPattern, replica.info.Collection, err)
  171. }
  172. if !matched {
  173. break
  174. }
  175. }
  176. collectionIsMismatch := false
  177. for _, volumeReplica := range replicas {
  178. if volumeReplica.info.Collection != replica.info.Collection {
  179. fmt.Fprintf(writer, "skip delete volume %d as collection %s is mismatch: %s\n", replica.info.Id, replica.info.Collection, volumeReplica.info.Collection)
  180. collectionIsMismatch = true
  181. }
  182. }
  183. if collectionIsMismatch {
  184. continue
  185. }
  186. fmt.Fprintf(writer, "deleting volume %d from %s ...\n", replica.info.Id, replica.location.dataNode.Id)
  187. if !takeAction {
  188. break
  189. }
  190. if err := deleteVolume(commandEnv.option.GrpcDialOption, needle.VolumeId(replica.info.Id),
  191. pb.NewServerAddressFromDataNode(replica.location.dataNode), false); err != nil {
  192. return fmt.Errorf("deleting volume %d from %s : %v", replica.info.Id, replica.location.dataNode.Id, err)
  193. }
  194. }
  195. return nil
  196. }
  197. func (c *commandVolumeFixReplication) fixUnderReplicatedVolumes(commandEnv *CommandEnv, writer io.Writer, takeAction bool, underReplicatedVolumeIds []uint32, volumeReplicas map[uint32][]*VolumeReplica, allLocations []location, retryCount int, volumesPerStep int) (fixedVolumes map[string]int, err error) {
  198. fixedVolumes = map[string]int{}
  199. if len(underReplicatedVolumeIds) > volumesPerStep && volumesPerStep > 0 {
  200. underReplicatedVolumeIds = underReplicatedVolumeIds[0:volumesPerStep]
  201. }
  202. for _, vid := range underReplicatedVolumeIds {
  203. for i := 0; i < retryCount+1; i++ {
  204. if err = c.fixOneUnderReplicatedVolume(commandEnv, writer, takeAction, volumeReplicas, vid, allLocations); err == nil {
  205. if takeAction {
  206. fixedVolumes[strconv.FormatUint(uint64(vid), 10)] = len(volumeReplicas[vid])
  207. }
  208. break
  209. } else {
  210. fmt.Fprintf(writer, "fixing under replicated volume %d: %v\n", vid, err)
  211. }
  212. }
  213. }
  214. return fixedVolumes, nil
  215. }
  216. func (c *commandVolumeFixReplication) fixOneUnderReplicatedVolume(commandEnv *CommandEnv, writer io.Writer, takeAction bool, volumeReplicas map[uint32][]*VolumeReplica, vid uint32, allLocations []location) error {
  217. replicas := volumeReplicas[vid]
  218. replica := pickOneReplicaToCopyFrom(replicas)
  219. replicaPlacement, _ := super_block.NewReplicaPlacementFromByte(byte(replica.info.ReplicaPlacement))
  220. foundNewLocation := false
  221. hasSkippedCollection := false
  222. keepDataNodesSorted(allLocations, types.ToDiskType(replica.info.DiskType))
  223. fn := capacityByFreeVolumeCount(types.ToDiskType(replica.info.DiskType))
  224. for _, dst := range allLocations {
  225. // check whether data nodes satisfy the constraints
  226. if fn(dst.dataNode) > 0 && satisfyReplicaPlacement(replicaPlacement, replicas, dst) {
  227. // check collection name pattern
  228. if *c.collectionPattern != "" {
  229. matched, err := filepath.Match(*c.collectionPattern, replica.info.Collection)
  230. if err != nil {
  231. return fmt.Errorf("match pattern %s with collection %s: %v", *c.collectionPattern, replica.info.Collection, err)
  232. }
  233. if !matched {
  234. hasSkippedCollection = true
  235. break
  236. }
  237. }
  238. // ask the volume server to replicate the volume
  239. foundNewLocation = true
  240. fmt.Fprintf(writer, "replicating volume %d %s from %s to dataNode %s ...\n", replica.info.Id, replicaPlacement, replica.location.dataNode.Id, dst.dataNode.Id)
  241. if !takeAction {
  242. // adjust free volume count
  243. dst.dataNode.DiskInfos[replica.info.DiskType].FreeVolumeCount--
  244. break
  245. }
  246. err := operation.WithVolumeServerClient(false, pb.NewServerAddressFromDataNode(dst.dataNode), commandEnv.option.GrpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
  247. stream, replicateErr := volumeServerClient.VolumeCopy(context.Background(), &volume_server_pb.VolumeCopyRequest{
  248. VolumeId: replica.info.Id,
  249. SourceDataNode: string(pb.NewServerAddressFromDataNode(replica.location.dataNode)),
  250. })
  251. if replicateErr != nil {
  252. return fmt.Errorf("copying from %s => %s : %v", replica.location.dataNode.Id, dst.dataNode.Id, replicateErr)
  253. }
  254. for {
  255. resp, recvErr := stream.Recv()
  256. if recvErr != nil {
  257. if recvErr == io.EOF {
  258. break
  259. } else {
  260. return recvErr
  261. }
  262. }
  263. if resp.ProcessedBytes > 0 {
  264. fmt.Fprintf(writer, "volume %d processed %d bytes\n", replica.info.Id, resp.ProcessedBytes)
  265. }
  266. }
  267. return nil
  268. })
  269. if err != nil {
  270. return err
  271. }
  272. // adjust free volume count
  273. dst.dataNode.DiskInfos[replica.info.DiskType].FreeVolumeCount--
  274. break
  275. }
  276. }
  277. if !foundNewLocation && !hasSkippedCollection {
  278. fmt.Fprintf(writer, "failed to place volume %d replica as %s, existing:%+v\n", replica.info.Id, replicaPlacement, len(replicas))
  279. }
  280. return nil
  281. }
  282. func keepDataNodesSorted(dataNodes []location, diskType types.DiskType) {
  283. fn := capacityByFreeVolumeCount(diskType)
  284. slices.SortFunc(dataNodes, func(a, b location) bool {
  285. return fn(a.dataNode) > fn(b.dataNode)
  286. })
  287. }
  288. /*
  289. if on an existing data node {
  290. return false
  291. }
  292. if different from existing dcs {
  293. if lack on different dcs {
  294. return true
  295. }else{
  296. return false
  297. }
  298. }
  299. if not on primary dc {
  300. return false
  301. }
  302. if different from existing racks {
  303. if lack on different racks {
  304. return true
  305. }else{
  306. return false
  307. }
  308. }
  309. if not on primary rack {
  310. return false
  311. }
  312. if lacks on same rack {
  313. return true
  314. } else {
  315. return false
  316. }
  317. */
  318. func satisfyReplicaPlacement(replicaPlacement *super_block.ReplicaPlacement, replicas []*VolumeReplica, possibleLocation location) bool {
  319. existingDataCenters, _, existingDataNodes := countReplicas(replicas)
  320. if _, found := existingDataNodes[possibleLocation.String()]; found {
  321. // avoid duplicated volume on the same data node
  322. return false
  323. }
  324. primaryDataCenters, _ := findTopKeys(existingDataCenters)
  325. // ensure data center count is within limit
  326. if _, found := existingDataCenters[possibleLocation.DataCenter()]; !found {
  327. // different from existing dcs
  328. if len(existingDataCenters) < replicaPlacement.DiffDataCenterCount+1 {
  329. // lack on different dcs
  330. return true
  331. } else {
  332. // adding this would go over the different dcs limit
  333. return false
  334. }
  335. }
  336. // now this is same as one of the existing data center
  337. if !isAmong(possibleLocation.DataCenter(), primaryDataCenters) {
  338. // not on one of the primary dcs
  339. return false
  340. }
  341. // now this is one of the primary dcs
  342. primaryDcRacks := make(map[string]int)
  343. for _, replica := range replicas {
  344. if replica.location.DataCenter() != possibleLocation.DataCenter() {
  345. continue
  346. }
  347. primaryDcRacks[replica.location.Rack()] += 1
  348. }
  349. primaryRacks, _ := findTopKeys(primaryDcRacks)
  350. sameRackCount := primaryDcRacks[possibleLocation.Rack()]
  351. // ensure rack count is within limit
  352. if _, found := primaryDcRacks[possibleLocation.Rack()]; !found {
  353. // different from existing racks
  354. if len(primaryDcRacks) < replicaPlacement.DiffRackCount+1 {
  355. // lack on different racks
  356. return true
  357. } else {
  358. // adding this would go over the different racks limit
  359. return false
  360. }
  361. }
  362. // now this is same as one of the existing racks
  363. if !isAmong(possibleLocation.Rack(), primaryRacks) {
  364. // not on the primary rack
  365. return false
  366. }
  367. // now this is on the primary rack
  368. // different from existing data nodes
  369. if sameRackCount < replicaPlacement.SameRackCount+1 {
  370. // lack on same rack
  371. return true
  372. } else {
  373. // adding this would go over the same data node limit
  374. return false
  375. }
  376. }
  377. func findTopKeys(m map[string]int) (topKeys []string, max int) {
  378. for k, c := range m {
  379. if max < c {
  380. topKeys = topKeys[:0]
  381. topKeys = append(topKeys, k)
  382. max = c
  383. } else if max == c {
  384. topKeys = append(topKeys, k)
  385. }
  386. }
  387. return
  388. }
  389. func isAmong(key string, keys []string) bool {
  390. for _, k := range keys {
  391. if k == key {
  392. return true
  393. }
  394. }
  395. return false
  396. }
  397. type VolumeReplica struct {
  398. location *location
  399. info *master_pb.VolumeInformationMessage
  400. }
  401. type location struct {
  402. dc string
  403. rack string
  404. dataNode *master_pb.DataNodeInfo
  405. }
  406. func newLocation(dc, rack string, dataNode *master_pb.DataNodeInfo) location {
  407. return location{
  408. dc: dc,
  409. rack: rack,
  410. dataNode: dataNode,
  411. }
  412. }
  413. func (l location) String() string {
  414. return fmt.Sprintf("%s %s %s", l.dc, l.rack, l.dataNode.Id)
  415. }
  416. func (l location) Rack() string {
  417. return fmt.Sprintf("%s %s", l.dc, l.rack)
  418. }
  419. func (l location) DataCenter() string {
  420. return l.dc
  421. }
  422. func pickOneReplicaToCopyFrom(replicas []*VolumeReplica) *VolumeReplica {
  423. mostRecent := replicas[0]
  424. for _, replica := range replicas {
  425. if replica.info.ModifiedAtSecond > mostRecent.info.ModifiedAtSecond {
  426. mostRecent = replica
  427. }
  428. }
  429. return mostRecent
  430. }
  431. func countReplicas(replicas []*VolumeReplica) (diffDc, diffRack, diffNode map[string]int) {
  432. diffDc = make(map[string]int)
  433. diffRack = make(map[string]int)
  434. diffNode = make(map[string]int)
  435. for _, replica := range replicas {
  436. diffDc[replica.location.DataCenter()] += 1
  437. diffRack[replica.location.Rack()] += 1
  438. diffNode[replica.location.String()] += 1
  439. }
  440. return
  441. }
  442. func pickOneReplicaToDelete(replicas []*VolumeReplica, replicaPlacement *super_block.ReplicaPlacement) *VolumeReplica {
  443. slices.SortFunc(replicas, func(a, b *VolumeReplica) bool {
  444. if a.info.Size != b.info.Size {
  445. return a.info.Size < b.info.Size
  446. }
  447. if a.info.ModifiedAtSecond != b.info.ModifiedAtSecond {
  448. return a.info.ModifiedAtSecond < b.info.ModifiedAtSecond
  449. }
  450. if a.info.CompactRevision != b.info.CompactRevision {
  451. return a.info.CompactRevision < b.info.CompactRevision
  452. }
  453. return false
  454. })
  455. return replicas[0]
  456. }
  457. // check and fix misplaced volumes
  458. func isMisplaced(replicas []*VolumeReplica, replicaPlacement *super_block.ReplicaPlacement) bool {
  459. for i := 0; i < len(replicas); i++ {
  460. others := otherThan(replicas, i)
  461. if satisfyReplicaPlacement(replicaPlacement, others, *replicas[i].location) {
  462. return false
  463. }
  464. }
  465. return true
  466. }
  467. func otherThan(replicas []*VolumeReplica, index int) (others []*VolumeReplica) {
  468. for i := 0; i < len(replicas); i++ {
  469. if index != i {
  470. others = append(others, replicas[i])
  471. }
  472. }
  473. return
  474. }
  475. func pickOneMisplacedVolume(replicas []*VolumeReplica, replicaPlacement *super_block.ReplicaPlacement) (toDelete *VolumeReplica) {
  476. var deletionCandidates []*VolumeReplica
  477. for i := 0; i < len(replicas); i++ {
  478. others := otherThan(replicas, i)
  479. if !isMisplaced(others, replicaPlacement) {
  480. deletionCandidates = append(deletionCandidates, replicas[i])
  481. }
  482. }
  483. if len(deletionCandidates) > 0 {
  484. return pickOneReplicaToDelete(deletionCandidates, replicaPlacement)
  485. }
  486. return pickOneReplicaToDelete(replicas, replicaPlacement)
  487. }