command_volume_fix_replication.go 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429
  1. package shell
  2. import (
  3. "context"
  4. "flag"
  5. "fmt"
  6. "github.com/chrislusf/seaweedfs/weed/storage/needle"
  7. "github.com/chrislusf/seaweedfs/weed/storage/types"
  8. "io"
  9. "path/filepath"
  10. "sort"
  11. "github.com/chrislusf/seaweedfs/weed/operation"
  12. "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
  13. "github.com/chrislusf/seaweedfs/weed/pb/volume_server_pb"
  14. "github.com/chrislusf/seaweedfs/weed/storage/super_block"
  15. )
  16. func init() {
  17. Commands = append(Commands, &commandVolumeFixReplication{})
  18. }
  19. type commandVolumeFixReplication struct {
  20. collectionPattern *string
  21. }
  22. func (c *commandVolumeFixReplication) Name() string {
  23. return "volume.fix.replication"
  24. }
  25. func (c *commandVolumeFixReplication) Help() string {
  26. return `add replicas to volumes that are missing replicas
  27. This command finds all over-replicated volumes. If found, it will purge the oldest copies and stop.
  28. This command also finds all under-replicated volumes, and finds volume servers with free slots.
  29. If the free slots satisfy the replication requirement, the volume content is copied over and mounted.
  30. volume.fix.replication -n # do not take action
  31. volume.fix.replication # actually deleting or copying the volume files and mount the volume
  32. volume.fix.replication -collectionPattern=important* # fix any collections with prefix "important"
  33. Note:
  34. * each time this will only add back one replica for each volume id that is under replicated.
  35. If there are multiple replicas are missing, e.g. replica count is > 2, you may need to run this multiple times.
  36. * do not run this too quickly within seconds, since the new volume replica may take a few seconds
  37. to register itself to the master.
  38. `
  39. }
  40. func (c *commandVolumeFixReplication) Do(args []string, commandEnv *CommandEnv, writer io.Writer) (err error) {
  41. if err = commandEnv.confirmIsLocked(); err != nil {
  42. return
  43. }
  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. retryCount := volFixReplicationCommand.Int("retry", 0, "how many times to retry")
  48. if err = volFixReplicationCommand.Parse(args); err != nil {
  49. return nil
  50. }
  51. takeAction := !*skipChange
  52. // collect topology information
  53. topologyInfo, _, err := collectTopologyInfo(commandEnv)
  54. if err != nil {
  55. return err
  56. }
  57. // find all volumes that needs replication
  58. // collect all data nodes
  59. volumeReplicas, allLocations := collectVolumeReplicaLocations(topologyInfo)
  60. if len(allLocations) == 0 {
  61. return fmt.Errorf("no data nodes at all")
  62. }
  63. // find all under replicated volumes
  64. var underReplicatedVolumeIds, overReplicatedVolumeIds []uint32
  65. for vid, replicas := range volumeReplicas {
  66. replica := replicas[0]
  67. replicaPlacement, _ := super_block.NewReplicaPlacementFromByte(byte(replica.info.ReplicaPlacement))
  68. if replicaPlacement.GetCopyCount() > len(replicas) {
  69. underReplicatedVolumeIds = append(underReplicatedVolumeIds, vid)
  70. } else if replicaPlacement.GetCopyCount() < len(replicas) {
  71. overReplicatedVolumeIds = append(overReplicatedVolumeIds, vid)
  72. fmt.Fprintf(writer, "volume %d replication %s, but over replicated %+d\n", replica.info.Id, replicaPlacement, len(replicas))
  73. }
  74. }
  75. if len(overReplicatedVolumeIds) > 0 {
  76. return c.fixOverReplicatedVolumes(commandEnv, writer, takeAction, overReplicatedVolumeIds, volumeReplicas, allLocations)
  77. }
  78. if len(underReplicatedVolumeIds) == 0 {
  79. return nil
  80. }
  81. // find the most under populated data nodes
  82. return c.fixUnderReplicatedVolumes(commandEnv, writer, takeAction, underReplicatedVolumeIds, volumeReplicas, allLocations, *retryCount)
  83. }
  84. func collectVolumeReplicaLocations(topologyInfo *master_pb.TopologyInfo) (map[uint32][]*VolumeReplica, []location) {
  85. volumeReplicas := make(map[uint32][]*VolumeReplica)
  86. var allLocations []location
  87. eachDataNode(topologyInfo, func(dc string, rack RackId, dn *master_pb.DataNodeInfo) {
  88. loc := newLocation(dc, string(rack), dn)
  89. for _, diskInfo := range dn.DiskInfos {
  90. for _, v := range diskInfo.VolumeInfos {
  91. volumeReplicas[v.Id] = append(volumeReplicas[v.Id], &VolumeReplica{
  92. location: &loc,
  93. info: v,
  94. })
  95. }
  96. }
  97. allLocations = append(allLocations, loc)
  98. })
  99. return volumeReplicas, allLocations
  100. }
  101. func (c *commandVolumeFixReplication) fixOverReplicatedVolumes(commandEnv *CommandEnv, writer io.Writer, takeAction bool, overReplicatedVolumeIds []uint32, volumeReplicas map[uint32][]*VolumeReplica, allLocations []location) error {
  102. for _, vid := range overReplicatedVolumeIds {
  103. replicas := volumeReplicas[vid]
  104. replicaPlacement, _ := super_block.NewReplicaPlacementFromByte(byte(replicas[0].info.ReplicaPlacement))
  105. replica := pickOneReplicaToDelete(replicas, replicaPlacement)
  106. // check collection name pattern
  107. if *c.collectionPattern != "" {
  108. matched, err := filepath.Match(*c.collectionPattern, replica.info.Collection)
  109. if err != nil {
  110. return fmt.Errorf("match pattern %s with collection %s: %v", *c.collectionPattern, replica.info.Collection, err)
  111. }
  112. if !matched {
  113. break
  114. }
  115. }
  116. fmt.Fprintf(writer, "deleting volume %d from %s ...\n", replica.info.Id, replica.location.dataNode.Id)
  117. if !takeAction {
  118. break
  119. }
  120. if err := deleteVolume(commandEnv.option.GrpcDialOption, needle.VolumeId(replica.info.Id), replica.location.dataNode.Id); err != nil {
  121. return fmt.Errorf("deleting volume %d from %s : %v", replica.info.Id, replica.location.dataNode.Id, err)
  122. }
  123. }
  124. return nil
  125. }
  126. func (c *commandVolumeFixReplication) fixUnderReplicatedVolumes(commandEnv *CommandEnv, writer io.Writer, takeAction bool, underReplicatedVolumeIds []uint32, volumeReplicas map[uint32][]*VolumeReplica, allLocations []location, retryCount int) (err error) {
  127. for _, vid := range underReplicatedVolumeIds {
  128. for i := 0; i < retryCount+1; i++ {
  129. if err = c.fixOneUnderReplicatedVolume(commandEnv, writer, takeAction, volumeReplicas, vid, allLocations); err == nil {
  130. break
  131. }
  132. }
  133. }
  134. return
  135. }
  136. func (c *commandVolumeFixReplication) fixOneUnderReplicatedVolume(commandEnv *CommandEnv, writer io.Writer, takeAction bool, volumeReplicas map[uint32][]*VolumeReplica, vid uint32, allLocations []location) error {
  137. replicas := volumeReplicas[vid]
  138. replica := pickOneReplicaToCopyFrom(replicas)
  139. replicaPlacement, _ := super_block.NewReplicaPlacementFromByte(byte(replica.info.ReplicaPlacement))
  140. foundNewLocation := false
  141. hasSkippedCollection := false
  142. keepDataNodesSorted(allLocations, types.ToDiskType(replica.info.DiskType))
  143. fn := capacityByFreeVolumeCount(types.ToDiskType(replica.info.DiskType))
  144. for _, dst := range allLocations {
  145. // check whether data nodes satisfy the constraints
  146. if fn(dst.dataNode) > 0 && satisfyReplicaPlacement(replicaPlacement, replicas, dst) {
  147. // check collection name pattern
  148. if *c.collectionPattern != "" {
  149. matched, err := filepath.Match(*c.collectionPattern, replica.info.Collection)
  150. if err != nil {
  151. return fmt.Errorf("match pattern %s with collection %s: %v", *c.collectionPattern, replica.info.Collection, err)
  152. }
  153. if !matched {
  154. hasSkippedCollection = true
  155. break
  156. }
  157. }
  158. // ask the volume server to replicate the volume
  159. foundNewLocation = true
  160. fmt.Fprintf(writer, "replicating volume %d %s from %s to dataNode %s ...\n", replica.info.Id, replicaPlacement, replica.location.dataNode.Id, dst.dataNode.Id)
  161. if !takeAction {
  162. // adjust free volume count
  163. dst.dataNode.DiskInfos[replica.info.DiskType].FreeVolumeCount--
  164. break
  165. }
  166. err := operation.WithVolumeServerClient(dst.dataNode.Id, commandEnv.option.GrpcDialOption, func(volumeServerClient volume_server_pb.VolumeServerClient) error {
  167. _, replicateErr := volumeServerClient.VolumeCopy(context.Background(), &volume_server_pb.VolumeCopyRequest{
  168. VolumeId: replica.info.Id,
  169. SourceDataNode: replica.location.dataNode.Id,
  170. })
  171. if replicateErr != nil {
  172. return fmt.Errorf("copying from %s => %s : %v", replica.location.dataNode.Id, dst.dataNode.Id, replicateErr)
  173. }
  174. return nil
  175. })
  176. if err != nil {
  177. return err
  178. }
  179. // adjust free volume count
  180. dst.dataNode.DiskInfos[replica.info.DiskType].FreeVolumeCount--
  181. break
  182. }
  183. }
  184. if !foundNewLocation && !hasSkippedCollection {
  185. fmt.Fprintf(writer, "failed to place volume %d replica as %s, existing:%+v\n", replica.info.Id, replicaPlacement, len(replicas))
  186. }
  187. return nil
  188. }
  189. func keepDataNodesSorted(dataNodes []location, diskType types.DiskType) {
  190. fn := capacityByFreeVolumeCount(diskType)
  191. sort.Slice(dataNodes, func(i, j int) bool {
  192. return fn(dataNodes[i].dataNode) > fn(dataNodes[j].dataNode)
  193. })
  194. }
  195. /*
  196. if on an existing data node {
  197. return false
  198. }
  199. if different from existing dcs {
  200. if lack on different dcs {
  201. return true
  202. }else{
  203. return false
  204. }
  205. }
  206. if not on primary dc {
  207. return false
  208. }
  209. if different from existing racks {
  210. if lack on different racks {
  211. return true
  212. }else{
  213. return false
  214. }
  215. }
  216. if not on primary rack {
  217. return false
  218. }
  219. if lacks on same rack {
  220. return true
  221. } else {
  222. return false
  223. }
  224. */
  225. func satisfyReplicaPlacement(replicaPlacement *super_block.ReplicaPlacement, replicas []*VolumeReplica, possibleLocation location) bool {
  226. existingDataCenters, _, existingDataNodes := countReplicas(replicas)
  227. if _, found := existingDataNodes[possibleLocation.String()]; found {
  228. // avoid duplicated volume on the same data node
  229. return false
  230. }
  231. primaryDataCenters, _ := findTopKeys(existingDataCenters)
  232. // ensure data center count is within limit
  233. if _, found := existingDataCenters[possibleLocation.DataCenter()]; !found {
  234. // different from existing dcs
  235. if len(existingDataCenters) < replicaPlacement.DiffDataCenterCount+1 {
  236. // lack on different dcs
  237. return true
  238. } else {
  239. // adding this would go over the different dcs limit
  240. return false
  241. }
  242. }
  243. // now this is same as one of the existing data center
  244. if !isAmong(possibleLocation.DataCenter(), primaryDataCenters) {
  245. // not on one of the primary dcs
  246. return false
  247. }
  248. // now this is one of the primary dcs
  249. primaryDcRacks := make(map[string]int)
  250. for _, replica := range replicas {
  251. if replica.location.DataCenter() != possibleLocation.DataCenter() {
  252. continue
  253. }
  254. primaryDcRacks[replica.location.Rack()] += 1
  255. }
  256. primaryRacks, _ := findTopKeys(primaryDcRacks)
  257. sameRackCount := primaryDcRacks[possibleLocation.Rack()]
  258. // ensure rack count is within limit
  259. if _, found := primaryDcRacks[possibleLocation.Rack()]; !found {
  260. // different from existing racks
  261. if len(primaryDcRacks) < replicaPlacement.DiffRackCount+1 {
  262. // lack on different racks
  263. return true
  264. } else {
  265. // adding this would go over the different racks limit
  266. return false
  267. }
  268. }
  269. // now this is same as one of the existing racks
  270. if !isAmong(possibleLocation.Rack(), primaryRacks) {
  271. // not on the primary rack
  272. return false
  273. }
  274. // now this is on the primary rack
  275. // different from existing data nodes
  276. if sameRackCount < replicaPlacement.SameRackCount+1 {
  277. // lack on same rack
  278. return true
  279. } else {
  280. // adding this would go over the same data node limit
  281. return false
  282. }
  283. }
  284. func findTopKeys(m map[string]int) (topKeys []string, max int) {
  285. for k, c := range m {
  286. if max < c {
  287. topKeys = topKeys[:0]
  288. topKeys = append(topKeys, k)
  289. max = c
  290. } else if max == c {
  291. topKeys = append(topKeys, k)
  292. }
  293. }
  294. return
  295. }
  296. func isAmong(key string, keys []string) bool {
  297. for _, k := range keys {
  298. if k == key {
  299. return true
  300. }
  301. }
  302. return false
  303. }
  304. type VolumeReplica struct {
  305. location *location
  306. info *master_pb.VolumeInformationMessage
  307. }
  308. type location struct {
  309. dc string
  310. rack string
  311. dataNode *master_pb.DataNodeInfo
  312. }
  313. func newLocation(dc, rack string, dataNode *master_pb.DataNodeInfo) location {
  314. return location{
  315. dc: dc,
  316. rack: rack,
  317. dataNode: dataNode,
  318. }
  319. }
  320. func (l location) String() string {
  321. return fmt.Sprintf("%s %s %s", l.dc, l.rack, l.dataNode.Id)
  322. }
  323. func (l location) Rack() string {
  324. return fmt.Sprintf("%s %s", l.dc, l.rack)
  325. }
  326. func (l location) DataCenter() string {
  327. return l.dc
  328. }
  329. func pickOneReplicaToCopyFrom(replicas []*VolumeReplica) *VolumeReplica {
  330. mostRecent := replicas[0]
  331. for _, replica := range replicas {
  332. if replica.info.ModifiedAtSecond > mostRecent.info.ModifiedAtSecond {
  333. mostRecent = replica
  334. }
  335. }
  336. return mostRecent
  337. }
  338. func countReplicas(replicas []*VolumeReplica) (diffDc, diffRack, diffNode map[string]int) {
  339. diffDc = make(map[string]int)
  340. diffRack = make(map[string]int)
  341. diffNode = make(map[string]int)
  342. for _, replica := range replicas {
  343. diffDc[replica.location.DataCenter()] += 1
  344. diffRack[replica.location.Rack()] += 1
  345. diffNode[replica.location.String()] += 1
  346. }
  347. return
  348. }
  349. func pickOneReplicaToDelete(replicas []*VolumeReplica, replicaPlacement *super_block.ReplicaPlacement) *VolumeReplica {
  350. sort.Slice(replicas, func(i, j int) bool {
  351. a, b := replicas[i], replicas[j]
  352. if a.info.CompactRevision != b.info.CompactRevision {
  353. return a.info.CompactRevision < b.info.CompactRevision
  354. }
  355. if a.info.ModifiedAtSecond != b.info.ModifiedAtSecond {
  356. return a.info.ModifiedAtSecond < b.info.ModifiedAtSecond
  357. }
  358. if a.info.Size != b.info.Size {
  359. return a.info.Size < b.info.Size
  360. }
  361. return false
  362. })
  363. return replicas[0]
  364. }