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.

400 lines
13 KiB

3 years ago
3 years ago
4 years ago
6 years ago
5 years ago
6 years ago
6 years ago
3 years ago
3 years ago
3 years ago
3 years ago
4 years ago
6 years ago
  1. package weed_server
  2. import (
  3. "context"
  4. "errors"
  5. "fmt"
  6. "github.com/seaweedfs/seaweedfs/weed/cluster"
  7. "net"
  8. "sort"
  9. "time"
  10. "github.com/seaweedfs/seaweedfs/weed/pb"
  11. "github.com/seaweedfs/seaweedfs/weed/stats"
  12. "github.com/seaweedfs/seaweedfs/weed/storage/backend"
  13. "github.com/seaweedfs/seaweedfs/weed/util"
  14. "github.com/seaweedfs/raft"
  15. "google.golang.org/grpc/peer"
  16. "github.com/seaweedfs/seaweedfs/weed/glog"
  17. "github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
  18. "github.com/seaweedfs/seaweedfs/weed/storage/needle"
  19. "github.com/seaweedfs/seaweedfs/weed/topology"
  20. )
  21. func (ms *MasterServer) RegisterUuids(heartbeat *master_pb.Heartbeat) (duplicated_uuids []string, err error) {
  22. ms.Topo.UuidAccessLock.Lock()
  23. defer ms.Topo.UuidAccessLock.Unlock()
  24. key := fmt.Sprintf("%s:%d", heartbeat.Ip, heartbeat.Port)
  25. if ms.Topo.UuidMap == nil {
  26. ms.Topo.UuidMap = make(map[string][]string)
  27. }
  28. // find whether new uuid exists
  29. for k, v := range ms.Topo.UuidMap {
  30. sort.Strings(v)
  31. for _, id := range heartbeat.LocationUuids {
  32. index := sort.SearchStrings(v, id)
  33. if index < len(v) && v[index] == id {
  34. duplicated_uuids = append(duplicated_uuids, id)
  35. glog.Errorf("directory of %s on %s has been loaded", id, k)
  36. }
  37. }
  38. }
  39. if len(duplicated_uuids) > 0 {
  40. return duplicated_uuids, errors.New("volume: Duplicated volume directories were loaded")
  41. }
  42. ms.Topo.UuidMap[key] = heartbeat.LocationUuids
  43. glog.V(0).Infof("found new uuid:%v %v , %v", key, heartbeat.LocationUuids, ms.Topo.UuidMap)
  44. return nil, nil
  45. }
  46. func (ms *MasterServer) UnRegisterUuids(ip string, port int) {
  47. ms.Topo.UuidAccessLock.Lock()
  48. defer ms.Topo.UuidAccessLock.Unlock()
  49. key := fmt.Sprintf("%s:%d", ip, port)
  50. delete(ms.Topo.UuidMap, key)
  51. glog.V(0).Infof("remove volume server %v, online volume server: %v", key, ms.Topo.UuidMap)
  52. }
  53. func (ms *MasterServer) SendHeartbeat(stream master_pb.Seaweed_SendHeartbeatServer) error {
  54. var dn *topology.DataNode
  55. defer func() {
  56. if dn != nil {
  57. dn.Counter--
  58. if dn.Counter > 0 {
  59. glog.V(0).Infof("disconnect phantom volume server %s:%d remaining %d", dn.Ip, dn.Port, dn.Counter)
  60. return
  61. }
  62. message := &master_pb.VolumeLocation{
  63. Url: dn.Url(),
  64. PublicUrl: dn.PublicUrl,
  65. }
  66. for _, v := range dn.GetVolumes() {
  67. message.DeletedVids = append(message.DeletedVids, uint32(v.Id))
  68. }
  69. for _, s := range dn.GetEcShards() {
  70. message.DeletedVids = append(message.DeletedVids, uint32(s.VolumeId))
  71. }
  72. // if the volume server disconnects and reconnects quickly
  73. // the unregister and register can race with each other
  74. ms.Topo.UnRegisterDataNode(dn)
  75. glog.V(0).Infof("unregister disconnected volume server %s:%d", dn.Ip, dn.Port)
  76. ms.UnRegisterUuids(dn.Ip, dn.Port)
  77. if len(message.DeletedVids) > 0 {
  78. ms.broadcastToClients(&master_pb.KeepConnectedResponse{VolumeLocation: message})
  79. }
  80. }
  81. }()
  82. for {
  83. heartbeat, err := stream.Recv()
  84. if err != nil {
  85. if dn != nil {
  86. glog.Warningf("SendHeartbeat.Recv server %s:%d : %v", dn.Ip, dn.Port, err)
  87. } else {
  88. glog.Warningf("SendHeartbeat.Recv: %v", err)
  89. }
  90. stats.MasterReceivedHeartbeatCounter.WithLabelValues("error").Inc()
  91. return err
  92. }
  93. if ms.Topo.IsLeader() {
  94. ms.Topo.Sequence.SetMax(heartbeat.MaxFileKey)
  95. if dn == nil {
  96. dcName, rackName := ms.Topo.Configuration.Locate(heartbeat.Ip, heartbeat.DataCenter, heartbeat.Rack)
  97. dc := ms.Topo.GetOrCreateDataCenter(dcName)
  98. rack := dc.GetOrCreateRack(rackName)
  99. dn = rack.GetOrCreateDataNode(heartbeat.Ip, int(heartbeat.Port), int(heartbeat.GrpcPort), heartbeat.PublicUrl, heartbeat.MaxVolumeCounts)
  100. glog.V(0).Infof("added volume server %d: %v:%d %v", dn.Counter, heartbeat.GetIp(), heartbeat.GetPort(), heartbeat.LocationUuids)
  101. uuidlist, err := ms.RegisterUuids(heartbeat)
  102. if err != nil {
  103. if stream_err := stream.Send(&master_pb.HeartbeatResponse{
  104. DuplicatedUuids: uuidlist,
  105. }); stream_err != nil {
  106. glog.Warningf("SendHeartbeat.Send DuplicatedDirectory response to %s:%d %v", dn.Ip, dn.Port, stream_err)
  107. return stream_err
  108. }
  109. return err
  110. }
  111. if err := stream.Send(&master_pb.HeartbeatResponse{
  112. VolumeSizeLimit: uint64(ms.option.VolumeSizeLimitMB) * 1024 * 1024,
  113. }); err != nil {
  114. glog.Warningf("SendHeartbeat.Send volume size to %s:%d %v", dn.Ip, dn.Port, err)
  115. return err
  116. }
  117. stats.MasterReceivedHeartbeatCounter.WithLabelValues("dataNode").Inc()
  118. dn.Counter++
  119. }
  120. dn.AdjustMaxVolumeCounts(heartbeat.MaxVolumeCounts)
  121. glog.V(4).Infof("master received heartbeat %s", heartbeat.String())
  122. stats.MasterReceivedHeartbeatCounter.WithLabelValues("total").Inc()
  123. message := &master_pb.VolumeLocation{
  124. Url: dn.Url(),
  125. PublicUrl: dn.PublicUrl,
  126. DataCenter: dn.GetDataCenterId(),
  127. }
  128. if len(heartbeat.NewVolumes) > 0 {
  129. stats.FilerRequestCounter.WithLabelValues("newVolumes").Inc()
  130. }
  131. if len(heartbeat.DeletedVolumes) > 0 {
  132. stats.FilerRequestCounter.WithLabelValues("deletedVolumes").Inc()
  133. }
  134. if len(heartbeat.NewVolumes) > 0 || len(heartbeat.DeletedVolumes) > 0 {
  135. // process delta volume ids if exists for fast volume id updates
  136. for _, volInfo := range heartbeat.NewVolumes {
  137. message.NewVids = append(message.NewVids, volInfo.Id)
  138. }
  139. for _, volInfo := range heartbeat.DeletedVolumes {
  140. message.DeletedVids = append(message.DeletedVids, volInfo.Id)
  141. }
  142. // update master internal volume layouts
  143. ms.Topo.IncrementalSyncDataNodeRegistration(heartbeat.NewVolumes, heartbeat.DeletedVolumes, dn)
  144. }
  145. if len(heartbeat.Volumes) > 0 || heartbeat.HasNoVolumes {
  146. dcName, rackName := ms.Topo.Configuration.Locate(heartbeat.Ip, heartbeat.DataCenter, heartbeat.Rack)
  147. ms.Topo.DataNodeRegistration(dcName, rackName, dn)
  148. // process heartbeat.Volumes
  149. stats.MasterReceivedHeartbeatCounter.WithLabelValues("Volumes").Inc()
  150. newVolumes, deletedVolumes := ms.Topo.SyncDataNodeRegistration(heartbeat.Volumes, dn)
  151. for _, v := range newVolumes {
  152. glog.V(0).Infof("master see new volume %d from %s", uint32(v.Id), dn.Url())
  153. message.NewVids = append(message.NewVids, uint32(v.Id))
  154. }
  155. for _, v := range deletedVolumes {
  156. glog.V(0).Infof("master see deleted volume %d from %s", uint32(v.Id), dn.Url())
  157. message.DeletedVids = append(message.DeletedVids, uint32(v.Id))
  158. }
  159. }
  160. if len(heartbeat.NewEcShards) > 0 || len(heartbeat.DeletedEcShards) > 0 {
  161. stats.MasterReceivedHeartbeatCounter.WithLabelValues("newEcShards").Inc()
  162. // update master internal volume layouts
  163. ms.Topo.IncrementalSyncDataNodeEcShards(heartbeat.NewEcShards, heartbeat.DeletedEcShards, dn)
  164. for _, s := range heartbeat.NewEcShards {
  165. message.NewEcVids = append(message.NewEcVids, s.Id)
  166. }
  167. for _, s := range heartbeat.DeletedEcShards {
  168. if dn.HasEcShards(needle.VolumeId(s.Id)) {
  169. continue
  170. }
  171. message.DeletedEcVids = append(message.DeletedEcVids, s.Id)
  172. }
  173. }
  174. if len(heartbeat.EcShards) > 0 || heartbeat.HasNoEcShards {
  175. stats.MasterReceivedHeartbeatCounter.WithLabelValues("ecShards").Inc()
  176. glog.V(4).Infof("master received ec shards from %s: %+v", dn.Url(), heartbeat.EcShards)
  177. newShards, deletedShards := ms.Topo.SyncDataNodeEcShards(heartbeat.EcShards, dn)
  178. // broadcast the ec vid changes to master clients
  179. for _, s := range newShards {
  180. message.NewEcVids = append(message.NewEcVids, uint32(s.VolumeId))
  181. }
  182. for _, s := range deletedShards {
  183. if dn.HasVolumesById(s.VolumeId) {
  184. continue
  185. }
  186. message.DeletedEcVids = append(message.DeletedEcVids, uint32(s.VolumeId))
  187. }
  188. }
  189. if len(message.NewVids) > 0 || len(message.DeletedVids) > 0 || len(message.NewEcVids) > 0 || len(message.DeletedEcVids) > 0 {
  190. ms.broadcastToClients(&master_pb.KeepConnectedResponse{VolumeLocation: message})
  191. }
  192. }
  193. // tell the volume servers about the leader
  194. newLeader, err := ms.Topo.Leader()
  195. if err != nil {
  196. glog.Warningf("SendHeartbeat find leader: %v", err)
  197. return err
  198. }
  199. if err := stream.Send(&master_pb.HeartbeatResponse{
  200. Leader: string(newLeader),
  201. }); err != nil {
  202. if dn != nil {
  203. glog.Warningf("SendHeartbeat.Send response to %s:%d %v", dn.Ip, dn.Port, err)
  204. } else {
  205. glog.Warningf("SendHeartbeat.Send response %v", err)
  206. }
  207. return err
  208. }
  209. }
  210. }
  211. // KeepConnected keep a stream gRPC call to the master. Used by clients to know the master is up.
  212. // And clients gets the up-to-date list of volume locations
  213. func (ms *MasterServer) KeepConnected(stream master_pb.Seaweed_KeepConnectedServer) error {
  214. req, recvErr := stream.Recv()
  215. if recvErr != nil {
  216. return recvErr
  217. }
  218. if !ms.Topo.IsLeader() {
  219. return ms.informNewLeader(stream)
  220. }
  221. peerAddress := pb.ServerAddress(req.ClientAddress)
  222. // buffer by 1 so we don't end up getting stuck writing to stopChan forever
  223. stopChan := make(chan bool, 1)
  224. clientName, messageChan := ms.addClient(req.FilerGroup, req.ClientType, peerAddress)
  225. for _, update := range ms.Cluster.AddClusterNode(req.FilerGroup, req.ClientType, cluster.DataCenter(req.DataCenter), cluster.Rack(req.Rack), peerAddress, req.Version) {
  226. ms.broadcastToClients(update)
  227. }
  228. defer func() {
  229. for _, update := range ms.Cluster.RemoveClusterNode(req.FilerGroup, req.ClientType, peerAddress) {
  230. ms.broadcastToClients(update)
  231. }
  232. ms.deleteClient(clientName)
  233. }()
  234. for i, message := range ms.Topo.ToVolumeLocations() {
  235. if i == 0 {
  236. if leader, err := ms.Topo.Leader(); err == nil {
  237. message.Leader = string(leader)
  238. }
  239. }
  240. if sendErr := stream.Send(&master_pb.KeepConnectedResponse{VolumeLocation: message}); sendErr != nil {
  241. return sendErr
  242. }
  243. }
  244. go func() {
  245. for {
  246. _, err := stream.Recv()
  247. if err != nil {
  248. glog.V(2).Infof("- client %v: %v", clientName, err)
  249. close(stopChan)
  250. return
  251. }
  252. }
  253. }()
  254. ticker := time.NewTicker(5 * time.Second)
  255. for {
  256. select {
  257. case message := <-messageChan:
  258. if err := stream.Send(message); err != nil {
  259. glog.V(0).Infof("=> client %v: %+v", clientName, message)
  260. return err
  261. }
  262. case <-ticker.C:
  263. if !ms.Topo.IsLeader() {
  264. stats.MasterRaftIsleader.Set(0)
  265. return ms.informNewLeader(stream)
  266. } else {
  267. stats.MasterRaftIsleader.Set(1)
  268. }
  269. case <-stopChan:
  270. return nil
  271. }
  272. }
  273. }
  274. func (ms *MasterServer) broadcastToClients(message *master_pb.KeepConnectedResponse) {
  275. ms.clientChansLock.RLock()
  276. for _, ch := range ms.clientChans {
  277. ch <- message
  278. }
  279. ms.clientChansLock.RUnlock()
  280. }
  281. func (ms *MasterServer) informNewLeader(stream master_pb.Seaweed_KeepConnectedServer) error {
  282. leader, err := ms.Topo.Leader()
  283. if err != nil {
  284. glog.Errorf("topo leader: %v", err)
  285. return raft.NotLeaderError
  286. }
  287. if err := stream.Send(&master_pb.KeepConnectedResponse{
  288. VolumeLocation: &master_pb.VolumeLocation{
  289. Leader: string(leader),
  290. },
  291. }); err != nil {
  292. return err
  293. }
  294. return nil
  295. }
  296. func (ms *MasterServer) addClient(filerGroup, clientType string, clientAddress pb.ServerAddress) (clientName string, messageChan chan *master_pb.KeepConnectedResponse) {
  297. clientName = filerGroup + "." + clientType + "@" + string(clientAddress)
  298. glog.V(0).Infof("+ client %v", clientName)
  299. // we buffer this because otherwise we end up in a potential deadlock where
  300. // the KeepConnected loop is no longer listening on this channel but we're
  301. // trying to send to it in SendHeartbeat and so we can't lock the
  302. // clientChansLock to remove the channel and we're stuck writing to it
  303. // 100 is probably overkill
  304. messageChan = make(chan *master_pb.KeepConnectedResponse, 100)
  305. ms.clientChansLock.Lock()
  306. ms.clientChans[clientName] = messageChan
  307. ms.clientChansLock.Unlock()
  308. return
  309. }
  310. func (ms *MasterServer) deleteClient(clientName string) {
  311. glog.V(0).Infof("- client %v", clientName)
  312. ms.clientChansLock.Lock()
  313. delete(ms.clientChans, clientName)
  314. ms.clientChansLock.Unlock()
  315. }
  316. func findClientAddress(ctx context.Context, grpcPort uint32) string {
  317. // fmt.Printf("FromContext %+v\n", ctx)
  318. pr, ok := peer.FromContext(ctx)
  319. if !ok {
  320. glog.Error("failed to get peer from ctx")
  321. return ""
  322. }
  323. if pr.Addr == net.Addr(nil) {
  324. glog.Error("failed to get peer address")
  325. return ""
  326. }
  327. if grpcPort == 0 {
  328. return pr.Addr.String()
  329. }
  330. if tcpAddr, ok := pr.Addr.(*net.TCPAddr); ok {
  331. externalIP := tcpAddr.IP
  332. return util.JoinHostPort(externalIP.String(), int(grpcPort))
  333. }
  334. return pr.Addr.String()
  335. }
  336. func (ms *MasterServer) GetMasterConfiguration(ctx context.Context, req *master_pb.GetMasterConfigurationRequest) (*master_pb.GetMasterConfigurationResponse, error) {
  337. // tell the volume servers about the leader
  338. leader, _ := ms.Topo.Leader()
  339. resp := &master_pb.GetMasterConfigurationResponse{
  340. MetricsAddress: ms.option.MetricsAddress,
  341. MetricsIntervalSeconds: uint32(ms.option.MetricsIntervalSec),
  342. StorageBackends: backend.ToPbStorageBackends(),
  343. DefaultReplication: ms.option.DefaultReplicaPlacement,
  344. VolumeSizeLimitMB: uint32(ms.option.VolumeSizeLimitMB),
  345. VolumePreallocate: ms.option.VolumePreallocate,
  346. Leader: string(leader),
  347. }
  348. return resp, nil
  349. }