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.

430 lines
13 KiB

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