broker_server.go 5.3 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169
  1. package broker
  2. import (
  3. "context"
  4. "github.com/seaweedfs/seaweedfs/weed/filer_client"
  5. "github.com/seaweedfs/seaweedfs/weed/glog"
  6. "github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
  7. "github.com/seaweedfs/seaweedfs/weed/mq/sub_coordinator"
  8. "github.com/seaweedfs/seaweedfs/weed/mq/topic"
  9. "sync"
  10. "time"
  11. "github.com/seaweedfs/seaweedfs/weed/cluster"
  12. "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
  13. "github.com/seaweedfs/seaweedfs/weed/wdclient"
  14. "google.golang.org/grpc"
  15. "github.com/seaweedfs/seaweedfs/weed/pb"
  16. "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
  17. "github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
  18. )
  19. type MessageQueueBrokerOption struct {
  20. Masters map[string]pb.ServerAddress
  21. FilerGroup string
  22. DataCenter string
  23. Rack string
  24. DefaultReplication string
  25. MaxMB int
  26. Ip string
  27. Port int
  28. Cipher bool
  29. VolumeServerAccess string // how to access volume servers
  30. }
  31. func (option *MessageQueueBrokerOption) BrokerAddress() pb.ServerAddress {
  32. return pb.NewServerAddress(option.Ip, option.Port, 0)
  33. }
  34. type MessageQueueBroker struct {
  35. mq_pb.UnimplementedSeaweedMessagingServer
  36. option *MessageQueueBrokerOption
  37. grpcDialOption grpc.DialOption
  38. MasterClient *wdclient.MasterClient
  39. filers map[pb.ServerAddress]struct{}
  40. currentFiler pb.ServerAddress
  41. localTopicManager *topic.LocalTopicManager
  42. PubBalancer *pub_balancer.PubBalancer
  43. lockAsBalancer *cluster.LiveLock
  44. SubCoordinator *sub_coordinator.SubCoordinator
  45. accessLock sync.Mutex
  46. fca *filer_client.FilerClientAccessor
  47. }
  48. func NewMessageBroker(option *MessageQueueBrokerOption, grpcDialOption grpc.DialOption) (mqBroker *MessageQueueBroker, err error) {
  49. pubBalancer := pub_balancer.NewPubBalancer()
  50. subCoordinator := sub_coordinator.NewSubCoordinator()
  51. mqBroker = &MessageQueueBroker{
  52. option: option,
  53. grpcDialOption: grpcDialOption,
  54. MasterClient: wdclient.NewMasterClient(grpcDialOption, option.FilerGroup, cluster.BrokerType, option.BrokerAddress(), option.DataCenter, option.Rack, *pb.NewServiceDiscoveryFromMap(option.Masters)),
  55. filers: make(map[pb.ServerAddress]struct{}),
  56. localTopicManager: topic.NewLocalTopicManager(),
  57. PubBalancer: pubBalancer,
  58. SubCoordinator: subCoordinator,
  59. }
  60. fca := &filer_client.FilerClientAccessor{
  61. GetFiler: mqBroker.GetFiler,
  62. GetGrpcDialOption: mqBroker.GetGrpcDialOption,
  63. }
  64. mqBroker.fca = fca
  65. subCoordinator.FilerClientAccessor = fca
  66. mqBroker.MasterClient.SetOnPeerUpdateFn(mqBroker.OnBrokerUpdate)
  67. pubBalancer.OnPartitionChange = mqBroker.SubCoordinator.OnPartitionChange
  68. go mqBroker.MasterClient.KeepConnectedToMaster(context.Background())
  69. existingNodes := cluster.ListExistingPeerUpdates(mqBroker.MasterClient.GetMaster(context.Background()), grpcDialOption, option.FilerGroup, cluster.FilerType)
  70. for _, newNode := range existingNodes {
  71. mqBroker.OnBrokerUpdate(newNode, time.Now())
  72. }
  73. // keep connecting to balancer
  74. go func() {
  75. for mqBroker.currentFiler == "" {
  76. time.Sleep(time.Millisecond * 237)
  77. }
  78. self := option.BrokerAddress()
  79. glog.V(0).Infof("broker %s found filer %s", self, mqBroker.currentFiler)
  80. newBrokerBalancerCh := make(chan string, 1)
  81. lockClient := cluster.NewLockClient(grpcDialOption, mqBroker.currentFiler)
  82. mqBroker.lockAsBalancer = lockClient.StartLongLivedLock(pub_balancer.LockBrokerBalancer, string(self), func(newLockOwner string) {
  83. glog.V(0).Infof("broker %s found balanacer %s", self, newLockOwner)
  84. newBrokerBalancerCh <- newLockOwner
  85. })
  86. mqBroker.KeepConnectedToBrokerBalancer(newBrokerBalancerCh)
  87. }()
  88. return mqBroker, nil
  89. }
  90. func (b *MessageQueueBroker) OnBrokerUpdate(update *master_pb.ClusterNodeUpdate, startFrom time.Time) {
  91. if update.NodeType != cluster.FilerType {
  92. return
  93. }
  94. address := pb.ServerAddress(update.Address)
  95. if update.IsAdd {
  96. b.filers[address] = struct{}{}
  97. if b.currentFiler == "" {
  98. b.currentFiler = address
  99. }
  100. } else {
  101. delete(b.filers, address)
  102. if b.currentFiler == address {
  103. for filer := range b.filers {
  104. b.currentFiler = filer
  105. break
  106. }
  107. }
  108. }
  109. }
  110. func (b *MessageQueueBroker) GetGrpcDialOption() grpc.DialOption {
  111. return b.grpcDialOption
  112. }
  113. func (b *MessageQueueBroker) GetFiler() pb.ServerAddress {
  114. return b.currentFiler
  115. }
  116. func (b *MessageQueueBroker) WithFilerClient(streamingMode bool, fn func(filer_pb.SeaweedFilerClient) error) error {
  117. return pb.WithFilerClient(streamingMode, 0, b.GetFiler(), b.grpcDialOption, fn)
  118. }
  119. func (b *MessageQueueBroker) AdjustedUrl(location *filer_pb.Location) string {
  120. return location.Url
  121. }
  122. func (b *MessageQueueBroker) GetDataCenter() string {
  123. return ""
  124. }
  125. func (b *MessageQueueBroker) withMasterClient(streamingMode bool, master pb.ServerAddress, fn func(client master_pb.SeaweedClient) error) error {
  126. return pb.WithMasterClient(streamingMode, master, b.grpcDialOption, false, func(client master_pb.SeaweedClient) error {
  127. return fn(client)
  128. })
  129. }
  130. func (b *MessageQueueBroker) withBrokerClient(streamingMode bool, server pb.ServerAddress, fn func(client mq_pb.SeaweedMessagingClient) error) error {
  131. return pb.WithBrokerGrpcClient(streamingMode, server.String(), b.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
  132. return fn(client)
  133. })
  134. }