receive broker stats

This commit is contained in:
chrislu
2023-09-14 23:49:05 -07:00
parent b771fefa37
commit 436d99443b
6 changed files with 888 additions and 386 deletions

View File

@@ -20,6 +20,9 @@ service SeaweedMessaging {
rpc CheckBrokerLoad (CheckBrokerLoadRequest) returns (CheckBrokerLoadResponse) {
}
// control plane for balancer
rpc ConnectToBalancer (stream ConnectToBalancerRequest) returns (stream ConnectToBalancerResponse) {
}
// control plane for topic partitions
rpc LookupTopicBrokers (LookupTopicBrokersRequest) returns (LookupTopicBrokersResponse) {
}
@@ -100,6 +103,26 @@ message CheckBrokerLoadResponse {
}
//////////////////////////////////////////////////
message BrokerStats {
int32 topic_partition_count = 1;
int64 message_count = 2;
int64 bytes_count = 3;
int32 cpu_usage_percent = 4;
}
message ConnectToBalancerRequest {
message InitMessage {
string broker = 1;
}
oneof message {
InitMessage init = 1;
BrokerStats stats = 2;
}
}
message ConnectToBalancerResponse {
string error = 1;
}
//////////////////////////////////////////////////
message LookupTopicBrokersRequest {
Topic topic = 1;
bool is_for_publish = 2;

File diff suppressed because it is too large Load Diff

View File

@@ -27,6 +27,8 @@ type SeaweedMessagingClient interface {
AssignSegmentBrokers(ctx context.Context, in *AssignSegmentBrokersRequest, opts ...grpc.CallOption) (*AssignSegmentBrokersResponse, error)
CheckSegmentStatus(ctx context.Context, in *CheckSegmentStatusRequest, opts ...grpc.CallOption) (*CheckSegmentStatusResponse, error)
CheckBrokerLoad(ctx context.Context, in *CheckBrokerLoadRequest, opts ...grpc.CallOption) (*CheckBrokerLoadResponse, error)
// control plane for balancer
ConnectToBalancer(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_ConnectToBalancerClient, error)
// control plane for topic partitions
LookupTopicBrokers(ctx context.Context, in *LookupTopicBrokersRequest, opts ...grpc.CallOption) (*LookupTopicBrokersResponse, error)
// a pub client will call this to get the topic partitions assignment
@@ -82,6 +84,37 @@ func (c *seaweedMessagingClient) CheckBrokerLoad(ctx context.Context, in *CheckB
return out, nil
}
func (c *seaweedMessagingClient) ConnectToBalancer(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_ConnectToBalancerClient, error) {
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[0], "/messaging_pb.SeaweedMessaging/ConnectToBalancer", opts...)
if err != nil {
return nil, err
}
x := &seaweedMessagingConnectToBalancerClient{stream}
return x, nil
}
type SeaweedMessaging_ConnectToBalancerClient interface {
Send(*ConnectToBalancerRequest) error
Recv() (*ConnectToBalancerResponse, error)
grpc.ClientStream
}
type seaweedMessagingConnectToBalancerClient struct {
grpc.ClientStream
}
func (x *seaweedMessagingConnectToBalancerClient) Send(m *ConnectToBalancerRequest) error {
return x.ClientStream.SendMsg(m)
}
func (x *seaweedMessagingConnectToBalancerClient) Recv() (*ConnectToBalancerResponse, error) {
m := new(ConnectToBalancerResponse)
if err := x.ClientStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
func (c *seaweedMessagingClient) LookupTopicBrokers(ctx context.Context, in *LookupTopicBrokersRequest, opts ...grpc.CallOption) (*LookupTopicBrokersResponse, error) {
out := new(LookupTopicBrokersResponse)
err := c.cc.Invoke(ctx, "/messaging_pb.SeaweedMessaging/LookupTopicBrokers", in, out, opts...)
@@ -119,7 +152,7 @@ func (c *seaweedMessagingClient) CheckTopicPartitionsStatus(ctx context.Context,
}
func (c *seaweedMessagingClient) Publish(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishClient, error) {
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[0], "/messaging_pb.SeaweedMessaging/Publish", opts...)
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[1], "/messaging_pb.SeaweedMessaging/Publish", opts...)
if err != nil {
return nil, err
}
@@ -150,7 +183,7 @@ func (x *seaweedMessagingPublishClient) Recv() (*PublishResponse, error) {
}
func (c *seaweedMessagingClient) Subscribe(ctx context.Context, in *SubscribeRequest, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeClient, error) {
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[1], "/messaging_pb.SeaweedMessaging/Subscribe", opts...)
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[2], "/messaging_pb.SeaweedMessaging/Subscribe", opts...)
if err != nil {
return nil, err
}
@@ -190,6 +223,8 @@ type SeaweedMessagingServer interface {
AssignSegmentBrokers(context.Context, *AssignSegmentBrokersRequest) (*AssignSegmentBrokersResponse, error)
CheckSegmentStatus(context.Context, *CheckSegmentStatusRequest) (*CheckSegmentStatusResponse, error)
CheckBrokerLoad(context.Context, *CheckBrokerLoadRequest) (*CheckBrokerLoadResponse, error)
// control plane for balancer
ConnectToBalancer(SeaweedMessaging_ConnectToBalancerServer) error
// control plane for topic partitions
LookupTopicBrokers(context.Context, *LookupTopicBrokersRequest) (*LookupTopicBrokersResponse, error)
// a pub client will call this to get the topic partitions assignment
@@ -218,6 +253,9 @@ func (UnimplementedSeaweedMessagingServer) CheckSegmentStatus(context.Context, *
func (UnimplementedSeaweedMessagingServer) CheckBrokerLoad(context.Context, *CheckBrokerLoadRequest) (*CheckBrokerLoadResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method CheckBrokerLoad not implemented")
}
func (UnimplementedSeaweedMessagingServer) ConnectToBalancer(SeaweedMessaging_ConnectToBalancerServer) error {
return status.Errorf(codes.Unimplemented, "method ConnectToBalancer not implemented")
}
func (UnimplementedSeaweedMessagingServer) LookupTopicBrokers(context.Context, *LookupTopicBrokersRequest) (*LookupTopicBrokersResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method LookupTopicBrokers not implemented")
}
@@ -321,6 +359,32 @@ func _SeaweedMessaging_CheckBrokerLoad_Handler(srv interface{}, ctx context.Cont
return interceptor(ctx, in, info, handler)
}
func _SeaweedMessaging_ConnectToBalancer_Handler(srv interface{}, stream grpc.ServerStream) error {
return srv.(SeaweedMessagingServer).ConnectToBalancer(&seaweedMessagingConnectToBalancerServer{stream})
}
type SeaweedMessaging_ConnectToBalancerServer interface {
Send(*ConnectToBalancerResponse) error
Recv() (*ConnectToBalancerRequest, error)
grpc.ServerStream
}
type seaweedMessagingConnectToBalancerServer struct {
grpc.ServerStream
}
func (x *seaweedMessagingConnectToBalancerServer) Send(m *ConnectToBalancerResponse) error {
return x.ServerStream.SendMsg(m)
}
func (x *seaweedMessagingConnectToBalancerServer) Recv() (*ConnectToBalancerRequest, error) {
m := new(ConnectToBalancerRequest)
if err := x.ServerStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
func _SeaweedMessaging_LookupTopicBrokers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(LookupTopicBrokersRequest)
if err := dec(in); err != nil {
@@ -481,6 +545,12 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{
},
},
Streams: []grpc.StreamDesc{
{
StreamName: "ConnectToBalancer",
Handler: _SeaweedMessaging_ConnectToBalancer_Handler,
ServerStreams: true,
ClientStreams: true,
},
{
StreamName: "Publish",
Handler: _SeaweedMessaging_Publish_Handler,