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.

596 lines
19 KiB

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