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.

383 lines
12 KiB

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