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.

645 lines
21 KiB

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