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
7 years ago
5 years ago
7 years ago
7 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. // tell the volume servers about the leader
  95. newLeader, err := ms.Topo.Leader()
  96. if err != nil {
  97. glog.Warningf("SendHeartbeat find leader: %v", err)
  98. return err
  99. }
  100. if err := stream.Send(&master_pb.HeartbeatResponse{
  101. Leader: string(newLeader),
  102. }); err != nil {
  103. if dn != nil {
  104. glog.Warningf("SendHeartbeat.Send response to %s:%d %v", dn.Ip, dn.Port, err)
  105. } else {
  106. glog.Warningf("SendHeartbeat.Send response %v", err)
  107. }
  108. return err
  109. }
  110. continue
  111. }
  112. ms.Topo.Sequence.SetMax(heartbeat.MaxFileKey)
  113. if dn == nil {
  114. dcName, rackName := ms.Topo.Configuration.Locate(heartbeat.Ip, heartbeat.DataCenter, heartbeat.Rack)
  115. dc := ms.Topo.GetOrCreateDataCenter(dcName)
  116. rack := dc.GetOrCreateRack(rackName)
  117. dn = rack.GetOrCreateDataNode(heartbeat.Ip, int(heartbeat.Port), int(heartbeat.GrpcPort), heartbeat.PublicUrl, heartbeat.MaxVolumeCounts)
  118. glog.V(0).Infof("added volume server %d: %v:%d %v", dn.Counter, heartbeat.GetIp(), heartbeat.GetPort(), heartbeat.LocationUuids)
  119. uuidlist, err := ms.RegisterUuids(heartbeat)
  120. if err != nil {
  121. if stream_err := stream.Send(&master_pb.HeartbeatResponse{
  122. DuplicatedUuids: uuidlist,
  123. }); stream_err != nil {
  124. glog.Warningf("SendHeartbeat.Send DuplicatedDirectory response to %s:%d %v", dn.Ip, dn.Port, stream_err)
  125. return stream_err
  126. }
  127. return err
  128. }
  129. if err := stream.Send(&master_pb.HeartbeatResponse{
  130. VolumeSizeLimit: uint64(ms.option.VolumeSizeLimitMB) * 1024 * 1024,
  131. }); err != nil {
  132. glog.Warningf("SendHeartbeat.Send volume size to %s:%d %v", dn.Ip, dn.Port, err)
  133. return err
  134. }
  135. stats.MasterReceivedHeartbeatCounter.WithLabelValues("dataNode").Inc()
  136. dn.Counter++
  137. }
  138. dn.AdjustMaxVolumeCounts(heartbeat.MaxVolumeCounts)
  139. glog.V(4).Infof("master received heartbeat %s", heartbeat.String())
  140. stats.MasterReceivedHeartbeatCounter.WithLabelValues("total").Inc()
  141. message := &master_pb.VolumeLocation{
  142. Url: dn.Url(),
  143. PublicUrl: dn.PublicUrl,
  144. DataCenter: dn.GetDataCenterId(),
  145. }
  146. if len(heartbeat.NewVolumes) > 0 {
  147. stats.FilerRequestCounter.WithLabelValues("newVolumes").Inc()
  148. }
  149. if len(heartbeat.DeletedVolumes) > 0 {
  150. stats.FilerRequestCounter.WithLabelValues("deletedVolumes").Inc()
  151. }
  152. if len(heartbeat.NewVolumes) > 0 || len(heartbeat.DeletedVolumes) > 0 {
  153. // process delta volume ids if exists for fast volume id updates
  154. for _, volInfo := range heartbeat.NewVolumes {
  155. message.NewVids = append(message.NewVids, volInfo.Id)
  156. }
  157. for _, volInfo := range heartbeat.DeletedVolumes {
  158. message.DeletedVids = append(message.DeletedVids, volInfo.Id)
  159. }
  160. // update master internal volume layouts
  161. ms.Topo.IncrementalSyncDataNodeRegistration(heartbeat.NewVolumes, heartbeat.DeletedVolumes, dn)
  162. }
  163. if len(heartbeat.Volumes) > 0 || heartbeat.HasNoVolumes {
  164. dcName, rackName := ms.Topo.Configuration.Locate(heartbeat.Ip, heartbeat.DataCenter, heartbeat.Rack)
  165. ms.Topo.DataNodeRegistration(dcName, rackName, dn)
  166. // process heartbeat.Volumes
  167. stats.MasterReceivedHeartbeatCounter.WithLabelValues("Volumes").Inc()
  168. newVolumes, deletedVolumes := ms.Topo.SyncDataNodeRegistration(heartbeat.Volumes, dn)
  169. for _, v := range newVolumes {
  170. glog.V(0).Infof("master see new volume %d from %s", uint32(v.Id), dn.Url())
  171. message.NewVids = append(message.NewVids, uint32(v.Id))
  172. }
  173. for _, v := range deletedVolumes {
  174. glog.V(0).Infof("master see deleted volume %d from %s", uint32(v.Id), dn.Url())
  175. message.DeletedVids = append(message.DeletedVids, uint32(v.Id))
  176. }
  177. }
  178. if len(heartbeat.NewEcShards) > 0 || len(heartbeat.DeletedEcShards) > 0 {
  179. stats.MasterReceivedHeartbeatCounter.WithLabelValues("newEcShards").Inc()
  180. // update master internal volume layouts
  181. ms.Topo.IncrementalSyncDataNodeEcShards(heartbeat.NewEcShards, heartbeat.DeletedEcShards, dn)
  182. for _, s := range heartbeat.NewEcShards {
  183. message.NewEcVids = append(message.NewEcVids, s.Id)
  184. }
  185. for _, s := range heartbeat.DeletedEcShards {
  186. if dn.HasEcShards(needle.VolumeId(s.Id)) {
  187. continue
  188. }
  189. message.DeletedEcVids = append(message.DeletedEcVids, s.Id)
  190. }
  191. }
  192. if len(heartbeat.EcShards) > 0 || heartbeat.HasNoEcShards {
  193. stats.MasterReceivedHeartbeatCounter.WithLabelValues("ecShards").Inc()
  194. glog.V(4).Infof("master received ec shards from %s: %+v", dn.Url(), heartbeat.EcShards)
  195. newShards, deletedShards := ms.Topo.SyncDataNodeEcShards(heartbeat.EcShards, dn)
  196. // broadcast the ec vid changes to master clients
  197. for _, s := range newShards {
  198. message.NewEcVids = append(message.NewEcVids, uint32(s.VolumeId))
  199. }
  200. for _, s := range deletedShards {
  201. if dn.HasVolumesById(s.VolumeId) {
  202. continue
  203. }
  204. message.DeletedEcVids = append(message.DeletedEcVids, uint32(s.VolumeId))
  205. }
  206. }
  207. if len(message.NewVids) > 0 || len(message.DeletedVids) > 0 || len(message.NewEcVids) > 0 || len(message.DeletedEcVids) > 0 {
  208. ms.broadcastToClients(&master_pb.KeepConnectedResponse{VolumeLocation: message})
  209. }
  210. }
  211. }
  212. // KeepConnected keep a stream gRPC call to the master. Used by clients to know the master is up.
  213. // And clients gets the up-to-date list of volume locations
  214. func (ms *MasterServer) KeepConnected(stream master_pb.Seaweed_KeepConnectedServer) error {
  215. req, recvErr := stream.Recv()
  216. if recvErr != nil {
  217. return recvErr
  218. }
  219. if !ms.Topo.IsLeader() {
  220. return ms.informNewLeader(stream)
  221. }
  222. peerAddress := pb.ServerAddress(req.ClientAddress)
  223. // buffer by 1 so we don't end up getting stuck writing to stopChan forever
  224. stopChan := make(chan bool, 1)
  225. clientName, messageChan := ms.addClient(req.FilerGroup, req.ClientType, peerAddress)
  226. for _, update := range ms.Cluster.AddClusterNode(req.FilerGroup, req.ClientType, cluster.DataCenter(req.DataCenter), cluster.Rack(req.Rack), peerAddress, req.Version) {
  227. ms.broadcastToClients(update)
  228. }
  229. defer func() {
  230. for _, update := range ms.Cluster.RemoveClusterNode(req.FilerGroup, req.ClientType, peerAddress) {
  231. ms.broadcastToClients(update)
  232. }
  233. ms.deleteClient(clientName)
  234. }()
  235. for i, message := range ms.Topo.ToVolumeLocations() {
  236. if i == 0 {
  237. if leader, err := ms.Topo.Leader(); err == nil {
  238. message.Leader = string(leader)
  239. }
  240. }
  241. if sendErr := stream.Send(&master_pb.KeepConnectedResponse{VolumeLocation: message}); sendErr != nil {
  242. return sendErr
  243. }
  244. }
  245. go func() {
  246. for {
  247. _, err := stream.Recv()
  248. if err != nil {
  249. glog.V(2).Infof("- client %v: %v", clientName, err)
  250. close(stopChan)
  251. return
  252. }
  253. }
  254. }()
  255. ticker := time.NewTicker(5 * time.Second)
  256. for {
  257. select {
  258. case message := <-messageChan:
  259. if err := stream.Send(message); err != nil {
  260. glog.V(0).Infof("=> client %v: %+v", clientName, message)
  261. return err
  262. }
  263. case <-ticker.C:
  264. if !ms.Topo.IsLeader() {
  265. stats.MasterRaftIsleader.Set(0)
  266. return ms.informNewLeader(stream)
  267. } else {
  268. stats.MasterRaftIsleader.Set(1)
  269. }
  270. case <-stopChan:
  271. return nil
  272. }
  273. }
  274. }
  275. func (ms *MasterServer) broadcastToClients(message *master_pb.KeepConnectedResponse) {
  276. ms.clientChansLock.RLock()
  277. for _, ch := range ms.clientChans {
  278. ch <- message
  279. }
  280. ms.clientChansLock.RUnlock()
  281. }
  282. func (ms *MasterServer) informNewLeader(stream master_pb.Seaweed_KeepConnectedServer) error {
  283. leader, err := ms.Topo.Leader()
  284. if err != nil {
  285. glog.Errorf("topo leader: %v", err)
  286. return raft.NotLeaderError
  287. }
  288. if err := stream.Send(&master_pb.KeepConnectedResponse{
  289. VolumeLocation: &master_pb.VolumeLocation{
  290. Leader: string(leader),
  291. },
  292. }); err != nil {
  293. return err
  294. }
  295. return nil
  296. }
  297. func (ms *MasterServer) addClient(filerGroup, clientType string, clientAddress pb.ServerAddress) (clientName string, messageChan chan *master_pb.KeepConnectedResponse) {
  298. clientName = filerGroup + "." + clientType + "@" + string(clientAddress)
  299. glog.V(0).Infof("+ client %v", clientName)
  300. // we buffer this because otherwise we end up in a potential deadlock where
  301. // the KeepConnected loop is no longer listening on this channel but we're
  302. // trying to send to it in SendHeartbeat and so we can't lock the
  303. // clientChansLock to remove the channel and we're stuck writing to it
  304. // 100 is probably overkill
  305. messageChan = make(chan *master_pb.KeepConnectedResponse, 100)
  306. ms.clientChansLock.Lock()
  307. ms.clientChans[clientName] = messageChan
  308. ms.clientChansLock.Unlock()
  309. return
  310. }
  311. func (ms *MasterServer) deleteClient(clientName string) {
  312. glog.V(0).Infof("- client %v", clientName)
  313. ms.clientChansLock.Lock()
  314. delete(ms.clientChans, clientName)
  315. ms.clientChansLock.Unlock()
  316. }
  317. func findClientAddress(ctx context.Context, grpcPort uint32) string {
  318. // fmt.Printf("FromContext %+v\n", ctx)
  319. pr, ok := peer.FromContext(ctx)
  320. if !ok {
  321. glog.Error("failed to get peer from ctx")
  322. return ""
  323. }
  324. if pr.Addr == net.Addr(nil) {
  325. glog.Error("failed to get peer address")
  326. return ""
  327. }
  328. if grpcPort == 0 {
  329. return pr.Addr.String()
  330. }
  331. if tcpAddr, ok := pr.Addr.(*net.TCPAddr); ok {
  332. externalIP := tcpAddr.IP
  333. return util.JoinHostPort(externalIP.String(), int(grpcPort))
  334. }
  335. return pr.Addr.String()
  336. }
  337. func (ms *MasterServer) GetMasterConfiguration(ctx context.Context, req *master_pb.GetMasterConfigurationRequest) (*master_pb.GetMasterConfigurationResponse, error) {
  338. // tell the volume servers about the leader
  339. leader, _ := ms.Topo.Leader()
  340. resp := &master_pb.GetMasterConfigurationResponse{
  341. MetricsAddress: ms.option.MetricsAddress,
  342. MetricsIntervalSeconds: uint32(ms.option.MetricsIntervalSec),
  343. StorageBackends: backend.ToPbStorageBackends(),
  344. DefaultReplication: ms.option.DefaultReplicaPlacement,
  345. VolumeSizeLimitMB: uint32(ms.option.VolumeSizeLimitMB),
  346. VolumePreallocate: ms.option.VolumePreallocate,
  347. Leader: string(leader),
  348. }
  349. return resp, nil
  350. }