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.

395 lines
13 KiB

6 years ago
6 years ago
6 years ago
6 years ago
6 years ago
4 years ago
6 years ago
5 years ago
6 years ago
6 years ago
6 years ago
  1. package storage
  2. import (
  3. "context"
  4. "fmt"
  5. "io"
  6. "os"
  7. "sort"
  8. "sync"
  9. "time"
  10. "github.com/klauspost/reedsolomon"
  11. "github.com/chrislusf/seaweedfs/weed/glog"
  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/stats"
  16. "github.com/chrislusf/seaweedfs/weed/storage/erasure_coding"
  17. "github.com/chrislusf/seaweedfs/weed/storage/needle"
  18. "github.com/chrislusf/seaweedfs/weed/storage/types"
  19. )
  20. func (s *Store) CollectErasureCodingHeartbeat() *master_pb.Heartbeat {
  21. var ecShardMessages []*master_pb.VolumeEcShardInformationMessage
  22. collectionEcShardSize := make(map[string]int64)
  23. for _, location := range s.Locations {
  24. location.ecVolumesLock.RLock()
  25. for _, ecShards := range location.ecVolumes {
  26. ecShardMessages = append(ecShardMessages, ecShards.ToVolumeEcShardInformationMessage()...)
  27. for _, ecShard := range ecShards.Shards {
  28. collectionEcShardSize[ecShards.Collection] += ecShard.Size()
  29. }
  30. }
  31. location.ecVolumesLock.RUnlock()
  32. }
  33. for col, size := range collectionEcShardSize {
  34. stats.VolumeServerDiskSizeGauge.WithLabelValues(col, "ec").Set(float64(size))
  35. }
  36. return &master_pb.Heartbeat{
  37. EcShards: ecShardMessages,
  38. HasNoEcShards: len(ecShardMessages) == 0,
  39. }
  40. }
  41. func (s *Store) MountEcShards(collection string, vid needle.VolumeId, shardId erasure_coding.ShardId) error {
  42. for _, location := range s.Locations {
  43. if err := location.LoadEcShard(collection, vid, shardId); err == nil {
  44. glog.V(0).Infof("MountEcShards %d.%d", vid, shardId)
  45. var shardBits erasure_coding.ShardBits
  46. s.NewEcShardsChan <- master_pb.VolumeEcShardInformationMessage{
  47. Id: uint32(vid),
  48. Collection: collection,
  49. EcIndexBits: uint32(shardBits.AddShardId(shardId)),
  50. DiskType: string(location.DiskType),
  51. }
  52. return nil
  53. } else if err == os.ErrNotExist {
  54. continue
  55. } else {
  56. return fmt.Errorf("%s load ec shard %d.%d: %v", location.Directory, vid, shardId, err)
  57. }
  58. }
  59. return fmt.Errorf("MountEcShards %d.%d not found on disk", vid, shardId)
  60. }
  61. func (s *Store) UnmountEcShards(vid needle.VolumeId, shardId erasure_coding.ShardId) error {
  62. ecShard, found := s.findEcShard(vid, shardId)
  63. if !found {
  64. return nil
  65. }
  66. var shardBits erasure_coding.ShardBits
  67. message := master_pb.VolumeEcShardInformationMessage{
  68. Id: uint32(vid),
  69. Collection: ecShard.Collection,
  70. EcIndexBits: uint32(shardBits.AddShardId(shardId)),
  71. DiskType: string(ecShard.DiskType),
  72. }
  73. for _, location := range s.Locations {
  74. if deleted := location.UnloadEcShard(vid, shardId); deleted {
  75. glog.V(0).Infof("UnmountEcShards %d.%d", vid, shardId)
  76. s.DeletedEcShardsChan <- message
  77. return nil
  78. }
  79. }
  80. return fmt.Errorf("UnmountEcShards %d.%d not found on disk", vid, shardId)
  81. }
  82. func (s *Store) findEcShard(vid needle.VolumeId, shardId erasure_coding.ShardId) (*erasure_coding.EcVolumeShard, bool) {
  83. for _, location := range s.Locations {
  84. if v, found := location.FindEcShard(vid, shardId); found {
  85. return v, found
  86. }
  87. }
  88. return nil, false
  89. }
  90. func (s *Store) FindEcVolume(vid needle.VolumeId) (*erasure_coding.EcVolume, bool) {
  91. for _, location := range s.Locations {
  92. if s, found := location.FindEcVolume(vid); found {
  93. return s, true
  94. }
  95. }
  96. return nil, false
  97. }
  98. func (s *Store) DestroyEcVolume(vid needle.VolumeId) {
  99. for _, location := range s.Locations {
  100. location.DestroyEcVolume(vid)
  101. }
  102. }
  103. func (s *Store) ReadEcShardNeedle(vid needle.VolumeId, n *needle.Needle, onReadSizeFn func(size types.Size)) (int, error) {
  104. for _, location := range s.Locations {
  105. if localEcVolume, found := location.FindEcVolume(vid); found {
  106. offset, size, intervals, err := localEcVolume.LocateEcShardNeedle(n.Id, localEcVolume.Version)
  107. if err != nil {
  108. return 0, fmt.Errorf("locate in local ec volume: %v", err)
  109. }
  110. if size.IsDeleted() {
  111. return 0, ErrorDeleted
  112. }
  113. if onReadSizeFn != nil {
  114. onReadSizeFn(size)
  115. }
  116. glog.V(3).Infof("read ec volume %d offset %d size %d intervals:%+v", vid, offset.ToActualOffset(), size, intervals)
  117. if len(intervals) > 1 {
  118. glog.V(3).Infof("ReadEcShardNeedle needle id %s intervals:%+v", n.String(), intervals)
  119. }
  120. bytes, isDeleted, err := s.readEcShardIntervals(vid, n.Id, localEcVolume, intervals)
  121. if err != nil {
  122. return 0, fmt.Errorf("ReadEcShardIntervals: %v", err)
  123. }
  124. if isDeleted {
  125. return 0, ErrorDeleted
  126. }
  127. err = n.ReadBytes(bytes, offset.ToActualOffset(), size, localEcVolume.Version)
  128. if err != nil {
  129. return 0, fmt.Errorf("readbytes: %v", err)
  130. }
  131. return len(bytes), nil
  132. }
  133. }
  134. return 0, fmt.Errorf("ec shard %d not found", vid)
  135. }
  136. func (s *Store) readEcShardIntervals(vid needle.VolumeId, needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, intervals []erasure_coding.Interval) (data []byte, is_deleted bool, err error) {
  137. if err = s.cachedLookupEcShardLocations(ecVolume); err != nil {
  138. return nil, false, fmt.Errorf("failed to locate shard via master grpc %s: %v", s.MasterAddress, err)
  139. }
  140. for i, interval := range intervals {
  141. if d, isDeleted, e := s.readOneEcShardInterval(needleId, ecVolume, interval); e != nil {
  142. return nil, isDeleted, e
  143. } else {
  144. if isDeleted {
  145. is_deleted = true
  146. }
  147. if i == 0 {
  148. data = d
  149. } else {
  150. data = append(data, d...)
  151. }
  152. }
  153. }
  154. return
  155. }
  156. func (s *Store) readOneEcShardInterval(needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, interval erasure_coding.Interval) (data []byte, is_deleted bool, err error) {
  157. shardId, actualOffset := interval.ToShardIdAndOffset(erasure_coding.ErasureCodingLargeBlockSize, erasure_coding.ErasureCodingSmallBlockSize)
  158. data = make([]byte, interval.Size)
  159. if shard, found := ecVolume.FindEcVolumeShard(shardId); found {
  160. if _, err = shard.ReadAt(data, actualOffset); err != nil {
  161. glog.V(0).Infof("read local ec shard %d.%d offset %d: %v", ecVolume.VolumeId, shardId, actualOffset, err)
  162. return
  163. }
  164. } else {
  165. ecVolume.ShardLocationsLock.RLock()
  166. sourceDataNodes, hasShardIdLocation := ecVolume.ShardLocations[shardId]
  167. ecVolume.ShardLocationsLock.RUnlock()
  168. // try reading directly
  169. if hasShardIdLocation {
  170. _, is_deleted, err = s.readRemoteEcShardInterval(sourceDataNodes, needleId, ecVolume.VolumeId, shardId, data, actualOffset)
  171. if err == nil {
  172. return
  173. }
  174. glog.V(0).Infof("clearing ec shard %d.%d locations: %v", ecVolume.VolumeId, shardId, err)
  175. }
  176. // try reading by recovering from other shards
  177. _, is_deleted, err = s.recoverOneRemoteEcShardInterval(needleId, ecVolume, shardId, data, actualOffset)
  178. if err == nil {
  179. return
  180. }
  181. glog.V(0).Infof("recover ec shard %d.%d : %v", ecVolume.VolumeId, shardId, err)
  182. }
  183. return
  184. }
  185. func forgetShardId(ecVolume *erasure_coding.EcVolume, shardId erasure_coding.ShardId) {
  186. // failed to access the source data nodes, clear it up
  187. ecVolume.ShardLocationsLock.Lock()
  188. delete(ecVolume.ShardLocations, shardId)
  189. ecVolume.ShardLocationsLock.Unlock()
  190. }
  191. func (s *Store) cachedLookupEcShardLocations(ecVolume *erasure_coding.EcVolume) (err error) {
  192. shardCount := len(ecVolume.ShardLocations)
  193. if shardCount < erasure_coding.DataShardsCount &&
  194. ecVolume.ShardLocationsRefreshTime.Add(11*time.Second).After(time.Now()) ||
  195. shardCount == erasure_coding.TotalShardsCount &&
  196. ecVolume.ShardLocationsRefreshTime.Add(37*time.Minute).After(time.Now()) ||
  197. shardCount >= erasure_coding.DataShardsCount &&
  198. ecVolume.ShardLocationsRefreshTime.Add(7*time.Minute).After(time.Now()) {
  199. // still fresh
  200. return nil
  201. }
  202. glog.V(3).Infof("lookup and cache ec volume %d locations", ecVolume.VolumeId)
  203. err = operation.WithMasterServerClient(s.MasterAddress, s.grpcDialOption, func(masterClient master_pb.SeaweedClient) error {
  204. req := &master_pb.LookupEcVolumeRequest{
  205. VolumeId: uint32(ecVolume.VolumeId),
  206. }
  207. resp, err := masterClient.LookupEcVolume(context.Background(), req)
  208. if err != nil {
  209. return fmt.Errorf("lookup ec volume %d: %v", ecVolume.VolumeId, err)
  210. }
  211. if len(resp.ShardIdLocations) < erasure_coding.DataShardsCount {
  212. return fmt.Errorf("only %d shards found but %d required", len(resp.ShardIdLocations), erasure_coding.DataShardsCount)
  213. }
  214. ecVolume.ShardLocationsLock.Lock()
  215. for _, shardIdLocations := range resp.ShardIdLocations {
  216. shardId := erasure_coding.ShardId(shardIdLocations.ShardId)
  217. delete(ecVolume.ShardLocations, shardId)
  218. for _, loc := range shardIdLocations.Locations {
  219. ecVolume.ShardLocations[shardId] = append(ecVolume.ShardLocations[shardId], loc.Url)
  220. }
  221. }
  222. ecVolume.ShardLocationsRefreshTime = time.Now()
  223. ecVolume.ShardLocationsLock.Unlock()
  224. return nil
  225. })
  226. return
  227. }
  228. func (s *Store) readRemoteEcShardInterval(sourceDataNodes []string, needleId types.NeedleId, vid needle.VolumeId, shardId erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
  229. if len(sourceDataNodes) == 0 {
  230. return 0, false, fmt.Errorf("failed to find ec shard %d.%d", vid, shardId)
  231. }
  232. for _, sourceDataNode := range sourceDataNodes {
  233. glog.V(3).Infof("read remote ec shard %d.%d from %s", vid, shardId, sourceDataNode)
  234. n, is_deleted, err = s.doReadRemoteEcShardInterval(sourceDataNode, needleId, vid, shardId, buf, offset)
  235. if err == nil {
  236. return
  237. }
  238. glog.V(1).Infof("read remote ec shard %d.%d from %s: %v", vid, shardId, sourceDataNode, err)
  239. }
  240. return
  241. }
  242. func (s *Store) doReadRemoteEcShardInterval(sourceDataNode string, needleId types.NeedleId, vid needle.VolumeId, shardId erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
  243. err = operation.WithVolumeServerClient(sourceDataNode, s.grpcDialOption, func(client volume_server_pb.VolumeServerClient) error {
  244. // copy data slice
  245. shardReadClient, err := client.VolumeEcShardRead(context.Background(), &volume_server_pb.VolumeEcShardReadRequest{
  246. VolumeId: uint32(vid),
  247. ShardId: uint32(shardId),
  248. Offset: offset,
  249. Size: int64(len(buf)),
  250. FileKey: uint64(needleId),
  251. })
  252. if err != nil {
  253. return fmt.Errorf("failed to start reading ec shard %d.%d from %s: %v", vid, shardId, sourceDataNode, err)
  254. }
  255. for {
  256. resp, receiveErr := shardReadClient.Recv()
  257. if receiveErr == io.EOF {
  258. break
  259. }
  260. if receiveErr != nil {
  261. return fmt.Errorf("receiving ec shard %d.%d from %s: %v", vid, shardId, sourceDataNode, receiveErr)
  262. }
  263. if resp.IsDeleted {
  264. is_deleted = true
  265. }
  266. copy(buf[n:n+len(resp.Data)], resp.Data)
  267. n += len(resp.Data)
  268. }
  269. return nil
  270. })
  271. if err != nil {
  272. return 0, is_deleted, fmt.Errorf("read ec shard %d.%d from %s: %v", vid, shardId, sourceDataNode, err)
  273. }
  274. return
  275. }
  276. func (s *Store) recoverOneRemoteEcShardInterval(needleId types.NeedleId, ecVolume *erasure_coding.EcVolume, shardIdToRecover erasure_coding.ShardId, buf []byte, offset int64) (n int, is_deleted bool, err error) {
  277. glog.V(3).Infof("recover ec shard %d.%d from other locations", ecVolume.VolumeId, shardIdToRecover)
  278. enc, err := reedsolomon.New(erasure_coding.DataShardsCount, erasure_coding.ParityShardsCount)
  279. if err != nil {
  280. return 0, false, fmt.Errorf("failed to create encoder: %v", err)
  281. }
  282. bufs := make([][]byte, erasure_coding.TotalShardsCount)
  283. var wg sync.WaitGroup
  284. ecVolume.ShardLocationsLock.RLock()
  285. for shardId, locations := range ecVolume.ShardLocations {
  286. // skip currnent shard or empty shard
  287. if shardId == shardIdToRecover {
  288. continue
  289. }
  290. if len(locations) == 0 {
  291. glog.V(3).Infof("readRemoteEcShardInterval missing %d.%d from %+v", ecVolume.VolumeId, shardId, locations)
  292. continue
  293. }
  294. // read from remote locations
  295. wg.Add(1)
  296. go func(shardId erasure_coding.ShardId, locations []string) {
  297. defer wg.Done()
  298. data := make([]byte, len(buf))
  299. nRead, isDeleted, readErr := s.readRemoteEcShardInterval(locations, needleId, ecVolume.VolumeId, shardId, data, offset)
  300. if readErr != nil {
  301. glog.V(3).Infof("recover: readRemoteEcShardInterval %d.%d %d bytes from %+v: %v", ecVolume.VolumeId, shardId, nRead, locations, readErr)
  302. forgetShardId(ecVolume, shardId)
  303. }
  304. if isDeleted {
  305. is_deleted = true
  306. }
  307. if nRead == len(buf) {
  308. bufs[shardId] = data
  309. }
  310. }(shardId, locations)
  311. }
  312. ecVolume.ShardLocationsLock.RUnlock()
  313. wg.Wait()
  314. if err = enc.ReconstructData(bufs); err != nil {
  315. glog.V(3).Infof("recovered ec shard %d.%d failed: %v", ecVolume.VolumeId, shardIdToRecover, err)
  316. return 0, false, err
  317. }
  318. glog.V(4).Infof("recovered ec shard %d.%d from other locations", ecVolume.VolumeId, shardIdToRecover)
  319. copy(buf, bufs[shardIdToRecover])
  320. return len(buf), is_deleted, nil
  321. }
  322. func (s *Store) EcVolumes() (ecVolumes []*erasure_coding.EcVolume) {
  323. for _, location := range s.Locations {
  324. location.ecVolumesLock.RLock()
  325. for _, v := range location.ecVolumes {
  326. ecVolumes = append(ecVolumes, v)
  327. }
  328. location.ecVolumesLock.RUnlock()
  329. }
  330. sort.Slice(ecVolumes, func(i, j int) bool {
  331. return ecVolumes[i].VolumeId > ecVolumes[j].VolumeId
  332. })
  333. return ecVolumes
  334. }