aboutsummaryrefslogtreecommitdiff
path: root/weed
diff options
context:
space:
mode:
Diffstat (limited to 'weed')
-rw-r--r--weed/mq/broker/broker_grpc_admin.go16
-rw-r--r--weed/mq/client/cmd/weed_sub/subscriber.go6
-rw-r--r--weed/mq/client/sub_client/connect_to_sub_coordinator.go49
-rw-r--r--weed/mq/client/sub_client/subscriber.go10
4 files changed, 43 insertions, 38 deletions
diff --git a/weed/mq/broker/broker_grpc_admin.go b/weed/mq/broker/broker_grpc_admin.go
index 1313d09ec..3c9ef282c 100644
--- a/weed/mq/broker/broker_grpc_admin.go
+++ b/weed/mq/broker/broker_grpc_admin.go
@@ -2,25 +2,21 @@ package broker
import (
"context"
- "github.com/seaweedfs/seaweedfs/weed/cluster"
- "github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
+ "github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
+ "github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
)
func (b *MessageQueueBroker) FindBrokerLeader(c context.Context, request *mq_pb.FindBrokerLeaderRequest) (*mq_pb.FindBrokerLeaderResponse, error) {
ret := &mq_pb.FindBrokerLeaderResponse{}
- err := b.withMasterClient(false, b.MasterClient.GetMaster(), func(client master_pb.SeaweedClient) error {
- resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
- ClientType: cluster.BrokerType,
- FilerGroup: request.FilerGroup,
+ err := b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
+ resp, err := client.FindLockOwner(context.Background(), &filer_pb.FindLockOwnerRequest{
+ Name: pub_balancer.LockBrokerBalancer,
})
if err != nil {
return err
}
- if len(resp.ClusterNodes) == 0 {
- return nil
- }
- ret.Broker = resp.ClusterNodes[0].Address
+ ret.Broker = resp.Owner
return nil
})
return ret, err
diff --git a/weed/mq/client/cmd/weed_sub/subscriber.go b/weed/mq/client/cmd/weed_sub/subscriber.go
index a175c948a..6d0e49be1 100644
--- a/weed/mq/client/cmd/weed_sub/subscriber.go
+++ b/weed/mq/client/cmd/weed_sub/subscriber.go
@@ -5,6 +5,7 @@ import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client"
+ "github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/util"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
@@ -14,7 +15,7 @@ import (
var (
namespace = flag.String("ns", "test", "namespace")
- topic = flag.String("topic", "test", "topic")
+ t = flag.String("topic", "test", "topic")
seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
clientId = flag.Uint("client_id", uint(util.RandomInt32()), "client id")
@@ -31,8 +32,7 @@ func main() {
}
contentConfig := &sub_client.ContentConfiguration{
- Namespace: *namespace,
- Topic: *topic,
+ Topic: topic.NewTopic(*namespace, *t),
Filter: "",
StartTime: time.Unix(1, 1),
}
diff --git a/weed/mq/client/sub_client/connect_to_sub_coordinator.go b/weed/mq/client/sub_client/connect_to_sub_coordinator.go
index 6d0eb0d43..4cc3c8ff2 100644
--- a/weed/mq/client/sub_client/connect_to_sub_coordinator.go
+++ b/weed/mq/client/sub_client/connect_to_sub_coordinator.go
@@ -15,15 +15,30 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
waitTime := 1 * time.Second
for {
for _, broker := range sub.bootstrapBrokers {
- // TODO find the balancer
+ // lookup topic brokers
+ var brokerLeader string
+ err := pb.WithBrokerGrpcClient(false, broker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
+ resp, err := client.FindBrokerLeader(context.Background(), &mq_pb.FindBrokerLeaderRequest{})
+ if err != nil {
+ return err
+ }
+ brokerLeader = resp.Broker
+ return nil
+ })
+ if err != nil {
+ glog.V(0).Infof("broker coordinator on %s: %v", broker, err)
+ continue
+ }
+ glog.V(0).Infof("found broker coordinator: %v", brokerLeader)
+
// connect to the balancer
- pb.WithBrokerGrpcClient(true, broker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
+ pb.WithBrokerGrpcClient(true, brokerLeader, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stream, err := client.SubscriberToSubCoordinator(ctx)
if err != nil {
- glog.V(0).Infof("subscriber %s/%s: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
+ glog.V(0).Infof("subscriber %s: %v", sub.ContentConfig.Topic, err)
return err
}
waitTime = 1 * time.Second
@@ -33,16 +48,13 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
if err := stream.Send(&mq_pb.SubscriberToSubCoordinatorRequest{
Message: &mq_pb.SubscriberToSubCoordinatorRequest_Init{
Init: &mq_pb.SubscriberToSubCoordinatorRequest_InitMessage{
- ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
+ ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
ConsumerGroupInstanceId: sub.SubscriberConfig.ConsumerGroupInstanceId,
- Topic: &mq_pb.Topic{
- Namespace: sub.ContentConfig.Namespace,
- Name: sub.ContentConfig.Topic,
- },
+ Topic: sub.ContentConfig.Topic.ToPbTopic(),
},
},
}); err != nil {
- glog.V(0).Infof("subscriber %s/%s send init: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
+ glog.V(0).Infof("subscriber %s send init: %v", sub.ContentConfig.Topic, err)
return err
}
@@ -50,12 +62,12 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
for {
resp, err := stream.Recv()
if err != nil {
- glog.V(0).Infof("subscriber %s/%s receive: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
+ glog.V(0).Infof("subscriber %s receive: %v", sub.ContentConfig.Topic, err)
return err
}
assignment := resp.GetAssignment()
if assignment != nil {
- glog.V(0).Infof("subscriber %s/%s receive assignment: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, assignment)
+ glog.V(0).Infof("subscriber %s receive assignment: %v", sub.ContentConfig.Topic, assignment)
}
sub.onEachAssignment(assignment)
}
@@ -63,7 +75,7 @@ func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
return nil
})
}
- glog.V(0).Infof("subscriber %s/%s/%s waiting for more assignments", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
+ glog.V(0).Infof("subscriber %s/%s waiting for more assignments", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
if waitTime < 10*time.Second {
waitTime += 1 * time.Second
}
@@ -85,10 +97,10 @@ func (sub *TopicSubscriber) onEachAssignment(assignment *mq_pb.SubscriberToSubCo
go func(partition *mq_pb.Partition, broker string) {
defer wg.Done()
defer func() { <-semaphore }()
- glog.V(0).Infof("subscriber %s/%s/%s assigned partition %+v at %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker)
+ glog.V(0).Infof("subscriber %s/%s assigned partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker)
err := sub.onEachPartition(partition, broker)
if err != nil {
- glog.V(0).Infof("subscriber %s/%s/%s partition %+v at %v: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker, err)
+ glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker, err)
}
}(assigned.Partition, assigned.Broker)
}
@@ -104,10 +116,7 @@ func (sub *TopicSubscriber) onEachPartition(partition *mq_pb.Partition, broker s
Init: &mq_pb.SubscribeMessageRequest_InitMessage{
ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
- Topic: &mq_pb.Topic{
- Namespace: sub.ContentConfig.Namespace,
- Name: sub.ContentConfig.Topic,
- },
+ Topic: sub.ContentConfig.Topic.ToPbTopic(),
PartitionOffset: &mq_pb.PartitionOffset{
Partition: partition,
StartTsNs: sub.alreadyProcessedTsNs,
@@ -122,7 +131,7 @@ func (sub *TopicSubscriber) onEachPartition(partition *mq_pb.Partition, broker s
return fmt.Errorf("create subscribe client: %v", err)
}
- glog.V(0).Infof("subscriber %s/%s/%s connected to partition %+v at %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker)
+ glog.V(0).Infof("subscriber %s/%s connected to partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker)
if sub.OnCompletionFunc != nil {
defer sub.OnCompletionFunc()
@@ -145,7 +154,7 @@ func (sub *TopicSubscriber) onEachPartition(partition *mq_pb.Partition, broker s
return fmt.Errorf("subscribe recv: %v", err)
}
if resp.Message == nil {
- glog.V(0).Infof("subscriber %s/%s/%s received nil message", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
+ glog.V(0).Infof("subscriber %s/%s received nil message", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
continue
}
switch m := resp.Message.(type) {
diff --git a/weed/mq/client/sub_client/subscriber.go b/weed/mq/client/sub_client/subscriber.go
index 9adc5197e..982c3f13b 100644
--- a/weed/mq/client/sub_client/subscriber.go
+++ b/weed/mq/client/sub_client/subscriber.go
@@ -1,6 +1,7 @@
package sub_client
import (
+ "github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc"
"time"
@@ -11,14 +12,13 @@ type SubscriberConfiguration struct {
ConsumerGroup string
ConsumerGroupInstanceId string
GroupMinimumPeers int32
- GroupMaximumPeers int32
- BootstrapServers []string
- GrpcDialOption grpc.DialOption
+ GroupMaximumPeers int32
+ BootstrapServers []string
+ GrpcDialOption grpc.DialOption
}
type ContentConfiguration struct {
- Namespace string
- Topic string
+ Topic topic.Topic
Filter string
StartTime time.Time
}