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.

286 lines
9.3 KiB

7 years ago
7 years ago
5 years ago
7 years ago
5 years ago
3 years ago
7 years ago
3 years ago
4 years ago
3 years ago
6 years ago
  1. package wdclient
  2. import (
  3. "context"
  4. "fmt"
  5. "github.com/chrislusf/seaweedfs/weed/stats"
  6. "math/rand"
  7. "time"
  8. "github.com/chrislusf/seaweedfs/weed/util"
  9. "google.golang.org/grpc"
  10. "github.com/chrislusf/seaweedfs/weed/glog"
  11. "github.com/chrislusf/seaweedfs/weed/pb"
  12. "github.com/chrislusf/seaweedfs/weed/pb/master_pb"
  13. )
  14. type MasterClient struct {
  15. FilerGroup string
  16. clientType string
  17. clientHost pb.ServerAddress
  18. rack string
  19. currentMaster pb.ServerAddress
  20. masters map[string]pb.ServerAddress
  21. grpcDialOption grpc.DialOption
  22. vidMap
  23. vidMapCacheSize int
  24. OnPeerUpdate func(update *master_pb.ClusterNodeUpdate, startFrom time.Time)
  25. }
  26. func NewMasterClient(grpcDialOption grpc.DialOption, filerGroup string, clientType string, clientHost pb.ServerAddress, clientDataCenter string, rack string, masters map[string]pb.ServerAddress) *MasterClient {
  27. return &MasterClient{
  28. FilerGroup: filerGroup,
  29. clientType: clientType,
  30. clientHost: clientHost,
  31. masters: masters,
  32. grpcDialOption: grpcDialOption,
  33. vidMap: newVidMap(clientDataCenter),
  34. vidMapCacheSize: 5,
  35. }
  36. }
  37. func (mc *MasterClient) GetLookupFileIdFunction() LookupFileIdFunctionType {
  38. return mc.LookupFileIdWithFallback
  39. }
  40. func (mc *MasterClient) LookupFileIdWithFallback(fileId string) (fullUrls []string, err error) {
  41. fullUrls, err = mc.vidMap.LookupFileId(fileId)
  42. if err == nil && len(fullUrls) > 0 {
  43. return
  44. }
  45. err = pb.WithMasterClient(false, mc.currentMaster, mc.grpcDialOption, func(client master_pb.SeaweedClient) error {
  46. resp, err := client.LookupVolume(context.Background(), &master_pb.LookupVolumeRequest{
  47. VolumeOrFileIds: []string{fileId},
  48. })
  49. if err != nil {
  50. return fmt.Errorf("LookupVolume failed: %v", err)
  51. }
  52. for vid, vidLocation := range resp.VolumeIdLocations {
  53. for _, vidLoc := range vidLocation.Locations {
  54. loc := Location{
  55. Url: vidLoc.Url,
  56. PublicUrl: vidLoc.PublicUrl,
  57. GrpcPort: int(vidLoc.GrpcPort),
  58. }
  59. mc.vidMap.addLocation(uint32(vid), loc)
  60. fullUrls = append(fullUrls, "http://"+loc.Url+"/"+fileId)
  61. }
  62. }
  63. return nil
  64. })
  65. return
  66. }
  67. func (mc *MasterClient) GetMaster() pb.ServerAddress {
  68. mc.WaitUntilConnected()
  69. return mc.currentMaster
  70. }
  71. func (mc *MasterClient) GetMasters() map[string]pb.ServerAddress {
  72. mc.WaitUntilConnected()
  73. return mc.masters
  74. }
  75. func (mc *MasterClient) WaitUntilConnected() {
  76. for mc.currentMaster == "" {
  77. time.Sleep(time.Duration(rand.Int31n(200)) * time.Millisecond)
  78. }
  79. }
  80. func (mc *MasterClient) KeepConnectedToMaster() {
  81. glog.V(1).Infof("%s.%s masterClient bootstraps with masters %v", mc.FilerGroup, mc.clientType, mc.masters)
  82. for {
  83. mc.tryAllMasters()
  84. time.Sleep(time.Second)
  85. }
  86. }
  87. func (mc *MasterClient) FindLeaderFromOtherPeers(myMasterAddress pb.ServerAddress) (leader string) {
  88. for _, master := range mc.masters {
  89. if master == myMasterAddress {
  90. continue
  91. }
  92. if grpcErr := pb.WithMasterClient(false, master, mc.grpcDialOption, func(client master_pb.SeaweedClient) error {
  93. ctx, cancel := context.WithTimeout(context.Background(), 120*time.Millisecond)
  94. defer cancel()
  95. resp, err := client.GetMasterConfiguration(ctx, &master_pb.GetMasterConfigurationRequest{})
  96. if err != nil {
  97. return err
  98. }
  99. leader = resp.Leader
  100. return nil
  101. }); grpcErr != nil {
  102. glog.V(0).Infof("connect to %s: %v", master, grpcErr)
  103. }
  104. if leader != "" {
  105. glog.V(0).Infof("existing leader is %s", leader)
  106. return
  107. }
  108. }
  109. glog.V(0).Infof("No existing leader found!")
  110. return
  111. }
  112. func (mc *MasterClient) tryAllMasters() {
  113. var nextHintedLeader pb.ServerAddress
  114. for _, master := range mc.masters {
  115. nextHintedLeader = mc.tryConnectToMaster(master)
  116. for nextHintedLeader != "" {
  117. nextHintedLeader = mc.tryConnectToMaster(nextHintedLeader)
  118. }
  119. mc.currentMaster = ""
  120. }
  121. }
  122. func (mc *MasterClient) tryConnectToMaster(master pb.ServerAddress) (nextHintedLeader pb.ServerAddress) {
  123. glog.V(1).Infof("%s.%s masterClient Connecting to master %v", mc.FilerGroup, mc.clientType, master)
  124. stats.MasterClientConnectCounter.WithLabelValues("total").Inc()
  125. gprcErr := pb.WithMasterClient(true, master, mc.grpcDialOption, func(client master_pb.SeaweedClient) error {
  126. ctx, cancel := context.WithCancel(context.Background())
  127. defer cancel()
  128. stream, err := client.KeepConnected(ctx)
  129. if err != nil {
  130. glog.V(1).Infof("%s.%s masterClient failed to keep connected to %s: %v", mc.FilerGroup, mc.clientType, master, err)
  131. stats.MasterClientConnectCounter.WithLabelValues(stats.FailedToKeepConnected).Inc()
  132. return err
  133. }
  134. if err = stream.Send(&master_pb.KeepConnectedRequest{
  135. FilerGroup: mc.FilerGroup,
  136. DataCenter: mc.DataCenter,
  137. Rack: mc.rack,
  138. ClientType: mc.clientType,
  139. ClientAddress: string(mc.clientHost),
  140. Version: util.Version(),
  141. }); err != nil {
  142. glog.V(0).Infof("%s.%s masterClient failed to send to %s: %v", mc.FilerGroup, mc.clientType, master, err)
  143. stats.MasterClientConnectCounter.WithLabelValues(stats.FailedToSend).Inc()
  144. return err
  145. }
  146. glog.V(1).Infof("%s.%s masterClient Connected to %v", mc.FilerGroup, mc.clientType, master)
  147. resp, err := stream.Recv()
  148. if err != nil {
  149. glog.V(0).Infof("%s.%s masterClient failed to receive from %s: %v", mc.FilerGroup, mc.clientType, master, err)
  150. stats.MasterClientConnectCounter.WithLabelValues(stats.FailedToReceive).Inc()
  151. return err
  152. }
  153. // check if it is the leader to determine whether to reset the vidMap
  154. if resp.VolumeLocation != nil {
  155. if resp.VolumeLocation.Leader != "" && string(master) != resp.VolumeLocation.Leader {
  156. glog.V(0).Infof("master %v redirected to leader %v", master, resp.VolumeLocation.Leader)
  157. nextHintedLeader = pb.ServerAddress(resp.VolumeLocation.Leader)
  158. stats.MasterClientConnectCounter.WithLabelValues(stats.RedirectedToleader).Inc()
  159. return nil
  160. }
  161. //mc.vidMap = newVidMap("")
  162. mc.resetVidMap()
  163. mc.updateVidMap(resp)
  164. } else {
  165. mc.resetVidMap()
  166. //mc.vidMap = newVidMap("")
  167. }
  168. mc.currentMaster = master
  169. for {
  170. resp, err := stream.Recv()
  171. if err != nil {
  172. glog.V(0).Infof("%s.%s masterClient failed to receive from %s: %v", mc.FilerGroup, mc.clientType, master, err)
  173. stats.MasterClientConnectCounter.WithLabelValues(stats.FailedToReceive).Inc()
  174. return err
  175. }
  176. if resp.VolumeLocation != nil {
  177. // maybe the leader is changed
  178. if resp.VolumeLocation.Leader != "" && string(mc.currentMaster) != resp.VolumeLocation.Leader {
  179. glog.V(0).Infof("currentMaster %v redirected to leader %v", mc.currentMaster, resp.VolumeLocation.Leader)
  180. nextHintedLeader = pb.ServerAddress(resp.VolumeLocation.Leader)
  181. stats.MasterClientConnectCounter.WithLabelValues(stats.RedirectedToleader).Inc()
  182. return nil
  183. }
  184. mc.updateVidMap(resp)
  185. }
  186. if resp.ClusterNodeUpdate != nil {
  187. update := resp.ClusterNodeUpdate
  188. if mc.OnPeerUpdate != nil {
  189. if update.FilerGroup == mc.FilerGroup {
  190. if update.IsAdd {
  191. glog.V(0).Infof("+ %s.%s %s leader:%v\n", update.FilerGroup, update.NodeType, update.Address, update.IsLeader)
  192. } else {
  193. glog.V(0).Infof("- %s.%s %s leader:%v\n", update.FilerGroup, update.NodeType, update.Address, update.IsLeader)
  194. }
  195. stats.MasterClientConnectCounter.WithLabelValues(stats.OnPeerUpdate).Inc()
  196. mc.OnPeerUpdate(update, time.Now())
  197. }
  198. }
  199. }
  200. }
  201. })
  202. if gprcErr != nil {
  203. stats.MasterClientConnectCounter.WithLabelValues(stats.Failed).Inc()
  204. glog.V(1).Infof("%s.%s masterClient failed to connect with master %v: %v", mc.FilerGroup, mc.clientType, master, gprcErr)
  205. }
  206. return
  207. }
  208. func (mc *MasterClient) updateVidMap(resp *master_pb.KeepConnectedResponse) {
  209. // process new volume location
  210. loc := Location{
  211. Url: resp.VolumeLocation.Url,
  212. PublicUrl: resp.VolumeLocation.PublicUrl,
  213. DataCenter: resp.VolumeLocation.DataCenter,
  214. GrpcPort: int(resp.VolumeLocation.GrpcPort),
  215. }
  216. for _, newVid := range resp.VolumeLocation.NewVids {
  217. glog.V(1).Infof("%s.%s: %s masterClient adds volume %d", mc.FilerGroup, mc.clientType, loc.Url, newVid)
  218. mc.addLocation(newVid, loc)
  219. }
  220. for _, deletedVid := range resp.VolumeLocation.DeletedVids {
  221. glog.V(1).Infof("%s.%s: %s masterClient removes volume %d", mc.FilerGroup, mc.clientType, loc.Url, deletedVid)
  222. mc.deleteLocation(deletedVid, loc)
  223. }
  224. for _, newEcVid := range resp.VolumeLocation.NewEcVids {
  225. glog.V(1).Infof("%s.%s: %s masterClient adds ec volume %d", mc.FilerGroup, mc.clientType, loc.Url, newEcVid)
  226. mc.addEcLocation(newEcVid, loc)
  227. }
  228. for _, deletedEcVid := range resp.VolumeLocation.DeletedEcVids {
  229. glog.V(1).Infof("%s.%s: %s masterClient removes ec volume %d", mc.FilerGroup, mc.clientType, loc.Url, deletedEcVid)
  230. mc.deleteEcLocation(deletedEcVid, loc)
  231. }
  232. }
  233. func (mc *MasterClient) WithClient(streamingMode bool, fn func(client master_pb.SeaweedClient) error) error {
  234. return util.Retry("master grpc", func() error {
  235. for mc.currentMaster == "" {
  236. time.Sleep(3 * time.Second)
  237. }
  238. return pb.WithMasterClient(streamingMode, mc.currentMaster, mc.grpcDialOption, func(client master_pb.SeaweedClient) error {
  239. return fn(client)
  240. })
  241. })
  242. }
  243. func (mc *MasterClient) resetVidMap() {
  244. tail := &vidMap{vid2Locations: mc.vid2Locations, ecVid2Locations: mc.ecVid2Locations, cache: mc.cache}
  245. mc.vidMap = newVidMap("")
  246. mc.vidMap.cache = tail
  247. for i := 0; i < mc.vidMapCacheSize && tail.cache != nil; i++ {
  248. if i == mc.vidMapCacheSize-1 {
  249. tail.cache = nil
  250. } else {
  251. tail = tail.cache
  252. }
  253. }
  254. }