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.

245 lines
6.6 KiB

7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
7 years ago
  1. package weed_server
  2. import (
  3. "fmt"
  4. "net"
  5. "strings"
  6. "time"
  7. "github.com/chrislusf/raft"
  8. "github.com/chrislusf/seaweedfs/weed/glog"
  9. "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
  10. "github.com/chrislusf/seaweedfs/weed/storage/needle"
  11. "github.com/chrislusf/seaweedfs/weed/topology"
  12. "google.golang.org/grpc/peer"
  13. )
  14. func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServer) error {
  15. var dn *topology.DataNode
  16. t := ms.Topo
  17. defer func() {
  18. if dn != nil {
  19. glog.V(0).Infof("unregister disconnected volume server %s:%d", dn.Ip, dn.Port)
  20. t.UnRegisterDataNode(dn)
  21. message := &master_pb.VolumeLocation{
  22. Url: dn.Url(),
  23. PublicUrl: dn.PublicUrl,
  24. }
  25. for _, v := range dn.GetVolumes() {
  26. message.DeletedVids = append(message.DeletedVids, uint32(v.Id))
  27. }
  28. for _, s := range dn.GetEcShards() {
  29. message.DeletedVids = append(message.DeletedVids, uint32(s.VolumeId))
  30. }
  31. if len(message.DeletedVids) > 0 {
  32. ms.clientChansLock.RLock()
  33. for _, ch := range ms.clientChans {
  34. ch <- message
  35. }
  36. ms.clientChansLock.RUnlock()
  37. }
  38. }
  39. }()
  40. for {
  41. heartbeat, err := stream.Recv()
  42. if err != nil {
  43. return err
  44. }
  45. if dn == nil {
  46. t.Sequence.SetMax(heartbeat.MaxFileKey)
  47. if heartbeat.Ip == "" {
  48. if pr, ok := peer.FromContext(stream.Context()); ok {
  49. if pr.Addr != net.Addr(nil) {
  50. heartbeat.Ip = pr.Addr.String()[0:strings.LastIndex(pr.Addr.String(), ":")]
  51. glog.V(0).Infof("remote IP address is detected as %v", heartbeat.Ip)
  52. }
  53. }
  54. }
  55. dcName, rackName := t.Configuration.Locate(heartbeat.Ip, heartbeat.DataCenter, heartbeat.Rack)
  56. dc := t.GetOrCreateDataCenter(dcName)
  57. rack := dc.GetOrCreateRack(rackName)
  58. dn = rack.GetOrCreateDataNode(heartbeat.Ip,
  59. int(heartbeat.Port), heartbeat.PublicUrl,
  60. int64(heartbeat.MaxVolumeCount))
  61. glog.V(0).Infof("added volume server %v:%d", heartbeat.GetIp(), heartbeat.GetPort())
  62. if err := stream.Send(&master_pb.HeartbeatResponse{
  63. VolumeSizeLimit: uint64(ms.volumeSizeLimitMB) * 1024 * 1024,
  64. }); err != nil {
  65. return err
  66. }
  67. }
  68. glog.V(4).Infof("master received heartbeat %s", heartbeat.String())
  69. message := &master_pb.VolumeLocation{
  70. Url: dn.Url(),
  71. PublicUrl: dn.PublicUrl,
  72. }
  73. if len(heartbeat.NewVolumes) > 0 || len(heartbeat.DeletedVolumes) > 0 {
  74. // process delta volume ids if exists for fast volume id updates
  75. for _, volInfo := range heartbeat.NewVolumes {
  76. message.NewVids = append(message.NewVids, volInfo.Id)
  77. }
  78. for _, volInfo := range heartbeat.DeletedVolumes {
  79. message.DeletedVids = append(message.DeletedVids, volInfo.Id)
  80. }
  81. // update master internal volume layouts
  82. t.IncrementalSyncDataNodeRegistration(heartbeat.NewVolumes, heartbeat.DeletedVolumes, dn)
  83. }
  84. if len(heartbeat.Volumes) > 0 {
  85. // process heartbeat.Volumes
  86. newVolumes, deletedVolumes := t.SyncDataNodeRegistration(heartbeat.Volumes, dn)
  87. for _, v := range newVolumes {
  88. glog.V(0).Infof("master see new volume %d from %s", uint32(v.Id), dn.Url())
  89. message.NewVids = append(message.NewVids, uint32(v.Id))
  90. }
  91. for _, v := range deletedVolumes {
  92. glog.V(0).Infof("master see deleted volume %d from %s", uint32(v.Id), dn.Url())
  93. message.DeletedVids = append(message.DeletedVids, uint32(v.Id))
  94. }
  95. }
  96. if len(heartbeat.NewEcShards) > 0 || len(heartbeat.DeletedEcShards) > 0 {
  97. // TODO send out the delta
  98. // update master internal volume layouts
  99. t.IncrementalSyncDataNodeEcShards(heartbeat.NewEcShards, heartbeat.DeletedEcShards, dn)
  100. for _, s := range heartbeat.NewEcShards {
  101. message.NewVids = append(message.NewVids, s.Id)
  102. }
  103. for _, s := range heartbeat.DeletedEcShards {
  104. if dn.HasVolumesById(needle.VolumeId(s.Id)) {
  105. continue
  106. }
  107. message.DeletedVids = append(message.DeletedVids, s.Id)
  108. }
  109. }
  110. if len(heartbeat.EcShards) > 0 {
  111. glog.V(0).Infof("master recieved ec shards from %s: %+v", dn.Url(), heartbeat.EcShards)
  112. newShards, deletedShards := t.SyncDataNodeEcShards(heartbeat.EcShards, dn)
  113. // broadcast the ec vid changes to master clients
  114. for _, s := range newShards {
  115. message.NewVids = append(message.NewVids, uint32(s.VolumeId))
  116. }
  117. for _, s := range deletedShards {
  118. if dn.HasVolumesById(s.VolumeId) {
  119. continue
  120. }
  121. message.DeletedVids = append(message.DeletedVids, uint32(s.VolumeId))
  122. }
  123. }
  124. if len(message.NewVids) > 0 || len(message.DeletedVids) > 0 {
  125. ms.clientChansLock.RLock()
  126. for host, ch := range ms.clientChans {
  127. glog.V(0).Infof("master send to %s: %s", host, message.String())
  128. ch <- message
  129. }
  130. ms.clientChansLock.RUnlock()
  131. }
  132. // tell the volume servers about the leader
  133. newLeader, err := t.Leader()
  134. if err != nil {
  135. return err
  136. }
  137. if err := stream.Send(&master_pb.HeartbeatResponse{
  138. Leader: newLeader,
  139. }); err != nil {
  140. return err
  141. }
  142. }
  143. }
  144. // KeepConnected keep a stream gRPC call to the master. Used by clients to know the master is up.
  145. // And clients gets the up-to-date list of volume locations
  146. func (ms *MasterServer) KeepConnected(stream master_pb.Seaweed_KeepConnectedServer) error {
  147. req, err := stream.Recv()
  148. if err != nil {
  149. return err
  150. }
  151. if !ms.Topo.IsLeader() {
  152. return raft.NotLeaderError
  153. }
  154. // remember client address
  155. ctx := stream.Context()
  156. // fmt.Printf("FromContext %+v\n", ctx)
  157. pr, ok := peer.FromContext(ctx)
  158. if !ok {
  159. glog.Error("failed to get peer from ctx")
  160. return fmt.Errorf("failed to get peer from ctx")
  161. }
  162. if pr.Addr == net.Addr(nil) {
  163. glog.Error("failed to get peer address")
  164. return fmt.Errorf("failed to get peer address")
  165. }
  166. clientName := req.Name + pr.Addr.String()
  167. glog.V(0).Infof("+ client %v", clientName)
  168. messageChan := make(chan *master_pb.VolumeLocation)
  169. stopChan := make(chan bool)
  170. ms.clientChansLock.Lock()
  171. ms.clientChans[clientName] = messageChan
  172. ms.clientChansLock.Unlock()
  173. defer func() {
  174. glog.V(0).Infof("- client %v", clientName)
  175. ms.clientChansLock.Lock()
  176. delete(ms.clientChans, clientName)
  177. ms.clientChansLock.Unlock()
  178. }()
  179. for _, message := range ms.Topo.ToVolumeLocations() {
  180. if err := stream.Send(message); err != nil {
  181. return err
  182. }
  183. }
  184. go func() {
  185. for {
  186. _, err := stream.Recv()
  187. if err != nil {
  188. glog.V(2).Infof("- client %v: %v", clientName, err)
  189. stopChan <- true
  190. break
  191. }
  192. }
  193. }()
  194. ticker := time.NewTicker(5 * time.Second)
  195. for {
  196. select {
  197. case message := <-messageChan:
  198. if err := stream.Send(message); err != nil {
  199. glog.V(0).Infof("=> client %v: %+v", clientName, message)
  200. return err
  201. }
  202. case <-ticker.C:
  203. if !ms.Topo.IsLeader() {
  204. return raft.NotLeaderError
  205. }
  206. case <-stopChan:
  207. return nil
  208. }
  209. }
  210. return nil
  211. }