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.

246 lines
6.7 KiB

7 years ago
7 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
6 years ago
6 years ago
6 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.option.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 || heartbeat.HasNoVolumes {
  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. // update master internal volume layouts
  98. t.IncrementalSyncDataNodeEcShards(heartbeat.NewEcShards, heartbeat.DeletedEcShards, dn)
  99. for _, s := range heartbeat.NewEcShards {
  100. message.NewVids = append(message.NewVids, s.Id)
  101. }
  102. for _, s := range heartbeat.DeletedEcShards {
  103. if dn.HasVolumesById(needle.VolumeId(s.Id)) {
  104. continue
  105. }
  106. message.DeletedVids = append(message.DeletedVids, s.Id)
  107. }
  108. }
  109. if len(heartbeat.EcShards) > 0 || heartbeat.HasNoEcShards {
  110. glog.V(1).Infof("master recieved ec shards from %s: %+v", dn.Url(), heartbeat.EcShards)
  111. newShards, deletedShards := t.SyncDataNodeEcShards(heartbeat.EcShards, dn)
  112. // broadcast the ec vid changes to master clients
  113. for _, s := range newShards {
  114. message.NewVids = append(message.NewVids, uint32(s.VolumeId))
  115. }
  116. for _, s := range deletedShards {
  117. if dn.HasVolumesById(s.VolumeId) {
  118. continue
  119. }
  120. message.DeletedVids = append(message.DeletedVids, uint32(s.VolumeId))
  121. }
  122. }
  123. if len(message.NewVids) > 0 || len(message.DeletedVids) > 0 {
  124. ms.clientChansLock.RLock()
  125. for host, ch := range ms.clientChans {
  126. glog.V(0).Infof("master send to %s: %s", host, message.String())
  127. ch <- message
  128. }
  129. ms.clientChansLock.RUnlock()
  130. }
  131. // tell the volume servers about the leader
  132. newLeader, err := t.Leader()
  133. if err != nil {
  134. return err
  135. }
  136. if err := stream.Send(&master_pb.HeartbeatResponse{
  137. Leader: newLeader,
  138. MetricsAddress: ms.option.MetricsAddress,
  139. MetricsIntervalSeconds: uint32(ms.option.MetricsIntervalSec),
  140. }); err != nil {
  141. return err
  142. }
  143. }
  144. }
  145. // KeepConnected keep a stream gRPC call to the master. Used by clients to know the master is up.
  146. // And clients gets the up-to-date list of volume locations
  147. func (ms *MasterServer) KeepConnected(stream master_pb.Seaweed_KeepConnectedServer) error {
  148. req, err := stream.Recv()
  149. if err != nil {
  150. return err
  151. }
  152. if !ms.Topo.IsLeader() {
  153. return raft.NotLeaderError
  154. }
  155. // remember client address
  156. ctx := stream.Context()
  157. // fmt.Printf("FromContext %+v\n", ctx)
  158. pr, ok := peer.FromContext(ctx)
  159. if !ok {
  160. glog.Error("failed to get peer from ctx")
  161. return fmt.Errorf("failed to get peer from ctx")
  162. }
  163. if pr.Addr == net.Addr(nil) {
  164. glog.Error("failed to get peer address")
  165. return fmt.Errorf("failed to get peer address")
  166. }
  167. clientName := req.Name + pr.Addr.String()
  168. glog.V(0).Infof("+ client %v", clientName)
  169. messageChan := make(chan *master_pb.VolumeLocation)
  170. stopChan := make(chan bool)
  171. ms.clientChansLock.Lock()
  172. ms.clientChans[clientName] = messageChan
  173. ms.clientChansLock.Unlock()
  174. defer func() {
  175. glog.V(0).Infof("- client %v", clientName)
  176. ms.clientChansLock.Lock()
  177. delete(ms.clientChans, clientName)
  178. ms.clientChansLock.Unlock()
  179. }()
  180. for _, message := range ms.Topo.ToVolumeLocations() {
  181. if err := stream.Send(message); err != nil {
  182. return err
  183. }
  184. }
  185. go func() {
  186. for {
  187. _, err := stream.Recv()
  188. if err != nil {
  189. glog.V(2).Infof("- client %v: %v", clientName, err)
  190. stopChan <- true
  191. break
  192. }
  193. }
  194. }()
  195. ticker := time.NewTicker(5 * time.Second)
  196. for {
  197. select {
  198. case message := <-messageChan:
  199. if err := stream.Send(message); err != nil {
  200. glog.V(0).Infof("=> client %v: %+v", clientName, message)
  201. return err
  202. }
  203. case <-ticker.C:
  204. if !ms.Topo.IsLeader() {
  205. return raft.NotLeaderError
  206. }
  207. case <-stopChan:
  208. return nil
  209. }
  210. }
  211. return nil
  212. }