Add message queue agent (#6463)
* scaffold message queue agent * adjust proto, add mq_agent * add agent client implementation * remove unused function * agent publish server implementation * adding agent
This commit is contained in:
@@ -6,12 +6,18 @@ import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"io"
|
||||
"reflect"
|
||||
"time"
|
||||
)
|
||||
|
||||
func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssignment, stopCh chan struct{}) error {
|
||||
type KeyedOffset struct {
|
||||
Key []byte
|
||||
Offset int64
|
||||
}
|
||||
|
||||
func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssignment, stopCh chan struct{}, onDataMessageFn OnDataMessageFn) error {
|
||||
// connect to the partition broker
|
||||
return pb.WithBrokerGrpcClient(true, assigned.LeaderBroker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
|
||||
|
||||
@@ -20,31 +26,30 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
return fmt.Errorf("create subscribe client: %v", err)
|
||||
}
|
||||
|
||||
perPartitionConcurrency := sub.SubscriberConfig.PerPartitionConcurrency
|
||||
if perPartitionConcurrency <= 0 {
|
||||
perPartitionConcurrency = 1
|
||||
slidingWindowSize := sub.SubscriberConfig.SlidingWindowSize
|
||||
if slidingWindowSize <= 0 {
|
||||
slidingWindowSize = 1
|
||||
}
|
||||
|
||||
var stopTsNs int64
|
||||
if !sub.ContentConfig.StopTime.IsZero() {
|
||||
stopTsNs = sub.ContentConfig.StopTime.UnixNano()
|
||||
po := findPartitionOffset(sub.ContentConfig.PartitionOffsets, assigned.Partition)
|
||||
if po == nil {
|
||||
po = &schema_pb.PartitionOffset{
|
||||
Partition: assigned.Partition,
|
||||
StartTsNs: time.Now().UnixNano(),
|
||||
StartType: schema_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
|
||||
}
|
||||
}
|
||||
|
||||
if err = subscribeClient.Send(&mq_pb.SubscribeMessageRequest{
|
||||
Message: &mq_pb.SubscribeMessageRequest_Init{
|
||||
Init: &mq_pb.SubscribeMessageRequest_InitMessage{
|
||||
ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
|
||||
ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
|
||||
Topic: sub.ContentConfig.Topic.ToPbTopic(),
|
||||
PartitionOffset: &mq_pb.PartitionOffset{
|
||||
Partition: assigned.Partition,
|
||||
StartTsNs: sub.ContentConfig.StartTime.UnixNano(),
|
||||
StopTsNs: stopTsNs,
|
||||
StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
|
||||
},
|
||||
Filter: sub.ContentConfig.Filter,
|
||||
FollowerBroker: assigned.FollowerBroker,
|
||||
Concurrency: perPartitionConcurrency,
|
||||
ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
|
||||
ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
|
||||
Topic: sub.ContentConfig.Topic.ToPbTopic(),
|
||||
PartitionOffset: po,
|
||||
Filter: sub.ContentConfig.Filter,
|
||||
FollowerBroker: assigned.FollowerBroker,
|
||||
SlidingWindowSize: slidingWindowSize,
|
||||
},
|
||||
},
|
||||
}); err != nil {
|
||||
@@ -57,24 +62,13 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
defer sub.OnCompletionFunc()
|
||||
}
|
||||
|
||||
type KeyedOffset struct {
|
||||
Key []byte
|
||||
Offset int64
|
||||
}
|
||||
|
||||
partitionOffsetChan := make(chan KeyedOffset, 1024)
|
||||
defer func() {
|
||||
close(partitionOffsetChan)
|
||||
}()
|
||||
executors := util.NewLimitedConcurrentExecutor(int(perPartitionConcurrency))
|
||||
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
case <-stopCh:
|
||||
subscribeClient.CloseSend()
|
||||
return
|
||||
case ack, ok := <-partitionOffsetChan:
|
||||
case ack, ok := <-sub.PartitionOffsetChan:
|
||||
if !ok {
|
||||
subscribeClient.CloseSend()
|
||||
return
|
||||
@@ -91,9 +85,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
}
|
||||
}()
|
||||
|
||||
var lastErr error
|
||||
|
||||
for lastErr == nil {
|
||||
for {
|
||||
// glog.V(0).Infof("subscriber %s/%s/%s waiting for message", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
|
||||
resp, err := subscribeClient.Recv()
|
||||
if err != nil {
|
||||
@@ -113,17 +105,7 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
fmt.Printf("empty key %+v, type %v\n", m, reflect.TypeOf(m))
|
||||
continue
|
||||
}
|
||||
executors.Execute(func() {
|
||||
processErr := sub.OnEachMessageFunc(m.Data.Key, m.Data.Value)
|
||||
if processErr == nil {
|
||||
partitionOffsetChan <- KeyedOffset{
|
||||
Key: m.Data.Key,
|
||||
Offset: m.Data.TsNs,
|
||||
}
|
||||
} else {
|
||||
lastErr = processErr
|
||||
}
|
||||
})
|
||||
onDataMessageFn(m)
|
||||
case *mq_pb.SubscribeMessageResponse_Ctrl:
|
||||
// glog.V(0).Infof("subscriber %s/%s/%s received control %+v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, m.Ctrl)
|
||||
if m.Ctrl.IsEndOfStream || m.Ctrl.IsEndOfTopic {
|
||||
@@ -132,6 +114,14 @@ func (sub *TopicSubscriber) onEachPartition(assigned *mq_pb.BrokerPartitionAssig
|
||||
}
|
||||
}
|
||||
|
||||
return lastErr
|
||||
})
|
||||
}
|
||||
|
||||
func findPartitionOffset(partitionOffsets []*schema_pb.PartitionOffset, partition *schema_pb.Partition) *schema_pb.PartitionOffset {
|
||||
for _, po := range partitionOffsets {
|
||||
if po.Partition == partition {
|
||||
return po
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -4,6 +4,7 @@ import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/glog"
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/util"
|
||||
"sync"
|
||||
"time"
|
||||
)
|
||||
@@ -20,6 +21,7 @@ func (sub *TopicSubscriber) Subscribe() error {
|
||||
go sub.startProcessors()
|
||||
|
||||
// loop forever
|
||||
// TODO shutdown the subscriber when not needed anymore
|
||||
sub.doKeepConnectedToSubCoordinator()
|
||||
|
||||
return nil
|
||||
@@ -66,7 +68,21 @@ func (sub *TopicSubscriber) startProcessors() {
|
||||
},
|
||||
},
|
||||
}
|
||||
err := sub.onEachPartition(assigned, stopChan)
|
||||
|
||||
executors := util.NewLimitedConcurrentExecutor(int(sub.SubscriberConfig.SlidingWindowSize))
|
||||
onDataMessageFn := func(m *mq_pb.SubscribeMessageResponse_Data) {
|
||||
executors.Execute(func() {
|
||||
processErr := sub.OnEachMessageFunc(m.Data.Key, m.Data.Value)
|
||||
if processErr == nil {
|
||||
sub.PartitionOffsetChan <- KeyedOffset{
|
||||
Key: m.Data.Key,
|
||||
Offset: m.Data.TsNs,
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
err := sub.onEachPartition(assigned, stopChan, onDataMessageFn)
|
||||
if err != nil {
|
||||
glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker, err)
|
||||
} else {
|
||||
|
||||
@@ -3,9 +3,9 @@ package sub_client
|
||||
import (
|
||||
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
|
||||
"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
|
||||
"google.golang.org/grpc"
|
||||
"sync"
|
||||
"time"
|
||||
)
|
||||
|
||||
type SubscriberConfiguration struct {
|
||||
@@ -14,16 +14,16 @@ type SubscriberConfiguration struct {
|
||||
ConsumerGroupInstanceId string
|
||||
GrpcDialOption grpc.DialOption
|
||||
MaxPartitionCount int32 // how many partitions to process concurrently
|
||||
PerPartitionConcurrency int32 // how many messages to process concurrently per partition
|
||||
SlidingWindowSize int32 // how many messages to process concurrently per partition
|
||||
}
|
||||
|
||||
type ContentConfiguration struct {
|
||||
Topic topic.Topic
|
||||
Filter string
|
||||
StartTime time.Time
|
||||
StopTime time.Time
|
||||
Topic topic.Topic
|
||||
Filter string
|
||||
PartitionOffsets []*schema_pb.PartitionOffset
|
||||
}
|
||||
|
||||
type OnDataMessageFn func(m *mq_pb.SubscribeMessageResponse_Data)
|
||||
type OnEachMessageFunc func(key, value []byte) (err error)
|
||||
type OnCompletionFunc func()
|
||||
|
||||
@@ -32,15 +32,17 @@ type TopicSubscriber struct {
|
||||
ContentConfig *ContentConfiguration
|
||||
brokerPartitionAssignmentChan chan *mq_pb.SubscriberToSubCoordinatorResponse
|
||||
brokerPartitionAssignmentAckChan chan *mq_pb.SubscriberToSubCoordinatorRequest
|
||||
OnDataMessageFnnc OnDataMessageFn
|
||||
OnEachMessageFunc OnEachMessageFunc
|
||||
OnCompletionFunc OnCompletionFunc
|
||||
bootstrapBrokers []string
|
||||
waitForMoreMessage bool
|
||||
activeProcessors map[topic.Partition]*ProcessorState
|
||||
activeProcessorsLock sync.Mutex
|
||||
PartitionOffsetChan chan KeyedOffset
|
||||
}
|
||||
|
||||
func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration) *TopicSubscriber {
|
||||
func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration, partitionOffsetChan chan KeyedOffset) *TopicSubscriber {
|
||||
return &TopicSubscriber{
|
||||
SubscriberConfig: subscriber,
|
||||
ContentConfig: content,
|
||||
@@ -49,6 +51,7 @@ func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfigu
|
||||
bootstrapBrokers: bootstrapBrokers,
|
||||
waitForMoreMessage: true,
|
||||
activeProcessors: make(map[topic.Partition]*ProcessorState),
|
||||
PartitionOffsetChan: partitionOffsetChan,
|
||||
}
|
||||
}
|
||||
|
||||
@@ -56,6 +59,10 @@ func (sub *TopicSubscriber) SetEachMessageFunc(onEachMessageFn OnEachMessageFunc
|
||||
sub.OnEachMessageFunc = onEachMessageFn
|
||||
}
|
||||
|
||||
func (sub *TopicSubscriber) SetOnDataMessageFn(fn OnDataMessageFn) {
|
||||
sub.OnDataMessageFnnc = fn
|
||||
}
|
||||
|
||||
func (sub *TopicSubscriber) SetCompletionFunc(onCompletionFn OnCompletionFunc) {
|
||||
sub.OnCompletionFunc = onCompletionFn
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user