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.

426 lines
13 KiB

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