Accumulated changes for message queue (#6600)
* rename * set agent address * refactor * add agent sub * pub messages * grpc new client * can publish records via agent * send init message with session id * fmt * check cancelled request while waiting * use sessionId * handle possible nil stream * subscriber process messages * separate debug port * use atomic int64 * less logs * minor * skip io.EOF * rename * remove unused * use saved offsets * do not reuse session, since always session id is new after restart remove last active ts from SessionEntry * simplify printing * purge unused * just proxy the subscription, skipping the session step * adjust offset types * subscribe offset type and possible value * start after the known tsns * avoid wrongly set startPosition * move * remove * refactor * typo * fix * fix changed path
This commit is contained in:
@@ -1,14 +0,0 @@
|
||||
package agent_client
|
||||
|
||||
import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
)
|
||||
|
||||
func (a *PublishSession) PublishMessageRecord(key []byte, record *schema_pb.RecordValue) error {
|
||||
return a.stream.Send(&mq_agent_pb.PublishRecordRequest{
|
||||
SessionId: a.sessionId,
|
||||
Key: key,
|
||||
Value: record,
|
||||
})
|
||||
}
|
||||
@@ -1,17 +0,0 @@
|
||||
package agent_client
|
||||
|
||||
import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
)
|
||||
|
||||
func (a *SubscribeSession) SubscribeMessageRecord(
|
||||
onEachMessageFn func(key []byte, record *schema_pb.RecordValue),
|
||||
onCompletionFn func()) error {
|
||||
for {
|
||||
resp, err := a.stream.Recv()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
onEachMessageFn(resp.Key, resp.Value)
|
||||
}
|
||||
}
|
||||
@@ -4,10 +4,10 @@ import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/schema"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
)
|
||||
|
||||
type PublishSession struct {
|
||||
@@ -15,13 +15,12 @@ type PublishSession struct {
|
||||
partitionCount int
|
||||
publisherName string
|
||||
stream grpc.BidiStreamingClient[mq_agent_pb.PublishRecordRequest, mq_agent_pb.PublishRecordResponse]
|
||||
sessionId int64
|
||||
}
|
||||
|
||||
func NewPublishSession(agentAddress string, topicSchema *schema.Schema, partitionCount int, publisherName string) (*PublishSession, error) {
|
||||
|
||||
// call local agent grpc server to create a new session
|
||||
clientConn, err := pb.GrpcDial(context.Background(), agentAddress, true, grpc.WithInsecure())
|
||||
clientConn, err := grpc.NewClient(agentAddress, grpc.WithTransportCredentials(insecure.NewCredentials()))
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("dial agent server %s: %v", agentAddress, err)
|
||||
}
|
||||
@@ -48,12 +47,17 @@ func NewPublishSession(agentAddress string, topicSchema *schema.Schema, partitio
|
||||
return nil, fmt.Errorf("publish record: %v", err)
|
||||
}
|
||||
|
||||
if err = stream.Send(&mq_agent_pb.PublishRecordRequest{
|
||||
SessionId: resp.SessionId,
|
||||
}); err != nil {
|
||||
return nil, fmt.Errorf("send session id: %v", err)
|
||||
}
|
||||
|
||||
return &PublishSession{
|
||||
schema: topicSchema,
|
||||
partitionCount: partitionCount,
|
||||
publisherName: publisherName,
|
||||
stream: stream,
|
||||
sessionId: resp.SessionId,
|
||||
}, nil
|
||||
}
|
||||
|
||||
@@ -68,3 +72,10 @@ func (a *PublishSession) CloseSession() error {
|
||||
a.schema = nil
|
||||
return err
|
||||
}
|
||||
|
||||
func (a *PublishSession) PublishMessageRecord(key []byte, record *schema_pb.RecordValue) error {
|
||||
return a.stream.Send(&mq_agent_pb.PublishRecordRequest{
|
||||
Key: key,
|
||||
Value: record,
|
||||
})
|
||||
}
|
||||
|
||||
@@ -4,50 +4,48 @@ import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_agent_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"google.golang.org/grpc"
|
||||
"google.golang.org/grpc/credentials/insecure"
|
||||
)
|
||||
|
||||
type SubscribeOption struct {
|
||||
ConsumerGroup string
|
||||
ConsumerGroupInstanceId string
|
||||
Topic topic.Topic
|
||||
OffsetType schema_pb.OffsetType
|
||||
OffsetTsNs int64
|
||||
Filter string
|
||||
MaxSubscribedPartitions int32
|
||||
PerPartitionConcurrency int32
|
||||
SlidingWindowSize int32
|
||||
}
|
||||
|
||||
type SubscribeSession struct {
|
||||
Option *SubscribeOption
|
||||
stream grpc.BidiStreamingClient[mq_agent_pb.SubscribeRecordRequest, mq_agent_pb.SubscribeRecordResponse]
|
||||
sessionId int64
|
||||
Option *SubscribeOption
|
||||
stream grpc.BidiStreamingClient[mq_agent_pb.SubscribeRecordRequest, mq_agent_pb.SubscribeRecordResponse]
|
||||
}
|
||||
|
||||
func NewSubscribeSession(agentAddress string, option *SubscribeOption) (*SubscribeSession, error) {
|
||||
// call local agent grpc server to create a new session
|
||||
clientConn, err := pb.GrpcDial(context.Background(), agentAddress, true, grpc.WithInsecure())
|
||||
clientConn, err := grpc.NewClient(agentAddress, grpc.WithTransportCredentials(insecure.NewCredentials()))
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("dial agent server %s: %v", agentAddress, err)
|
||||
}
|
||||
agentClient := mq_agent_pb.NewSeaweedMessagingAgentClient(clientConn)
|
||||
|
||||
resp, err := agentClient.StartSubscribeSession(context.Background(), &mq_agent_pb.StartSubscribeSessionRequest{
|
||||
initRequest := &mq_agent_pb.SubscribeRecordRequest_InitSubscribeRecordRequest{
|
||||
ConsumerGroup: option.ConsumerGroup,
|
||||
ConsumerGroupInstanceId: option.ConsumerGroupInstanceId,
|
||||
Topic: &schema_pb.Topic{
|
||||
Namespace: option.Topic.Namespace,
|
||||
Name: option.Topic.Name,
|
||||
},
|
||||
OffsetType: option.OffsetType,
|
||||
OffsetTsNs: option.OffsetTsNs,
|
||||
MaxSubscribedPartitions: option.MaxSubscribedPartitions,
|
||||
Filter: option.Filter,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if resp.Error != "" {
|
||||
return nil, fmt.Errorf("start subscribe session: %v", resp.Error)
|
||||
SlidingWindowSize: option.SlidingWindowSize,
|
||||
}
|
||||
|
||||
stream, err := agentClient.SubscribeRecord(context.Background())
|
||||
@@ -55,9 +53,35 @@ func NewSubscribeSession(agentAddress string, option *SubscribeOption) (*Subscri
|
||||
return nil, fmt.Errorf("subscribe record: %v", err)
|
||||
}
|
||||
|
||||
if err = stream.Send(&mq_agent_pb.SubscribeRecordRequest{
|
||||
Init: initRequest,
|
||||
}); err != nil {
|
||||
return nil, fmt.Errorf("send session id: %v", err)
|
||||
}
|
||||
|
||||
return &SubscribeSession{
|
||||
Option: option,
|
||||
stream: stream,
|
||||
sessionId: resp.SessionId,
|
||||
Option: option,
|
||||
stream: stream,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *SubscribeSession) CloseSession() error {
|
||||
err := s.stream.CloseSend()
|
||||
return err
|
||||
}
|
||||
|
||||
func (a *SubscribeSession) SubscribeMessageRecord(
|
||||
onEachMessageFn func(key []byte, record *schema_pb.RecordValue),
|
||||
onCompletionFn func()) error {
|
||||
for {
|
||||
resp, err := a.stream.Recv()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
onEachMessageFn(resp.Key, resp.Value)
|
||||
}
|
||||
if onCompletionFn != nil {
|
||||
onCompletionFn()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
Reference in New Issue
Block a user