mirror of
https://github.com/seaweedfs/seaweedfs.git
synced 2025-06-28 10:36:36 +08:00

Some checks are pending
go: build dev binaries / cleanup (push) Waiting to run
go: build dev binaries / build_dev_linux_windows (amd64, linux) (push) Blocked by required conditions
go: build dev binaries / build_dev_linux_windows (amd64, windows) (push) Blocked by required conditions
go: build dev binaries / build_dev_darwin (amd64, darwin) (push) Blocked by required conditions
go: build dev binaries / build_dev_darwin (arm64, darwin) (push) Blocked by required conditions
docker: build dev containers / build-dev-containers (push) Waiting to run
End to End / FUSE Mount (push) Waiting to run
go: build binary / Build (push) Waiting to run
Ceph S3 tests / Ceph S3 tests (push) Waiting to run
* 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
138 lines
4.5 KiB
Go
138 lines
4.5 KiB
Go
package sub_client
|
|
|
|
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"
|
|
)
|
|
|
|
type ProcessorState struct {
|
|
stopCh chan struct{}
|
|
}
|
|
|
|
// Subscribe subscribes to a topic's specified partitions.
|
|
// If a partition is moved to another broker, the subscriber will automatically reconnect to the new broker.
|
|
|
|
func (sub *TopicSubscriber) Subscribe() error {
|
|
|
|
go sub.startProcessors()
|
|
|
|
// loop forever
|
|
// TODO shutdown the subscriber when not needed anymore
|
|
sub.doKeepConnectedToSubCoordinator()
|
|
|
|
return nil
|
|
}
|
|
|
|
func (sub *TopicSubscriber) startProcessors() {
|
|
// listen to the messages from the sub coordinator
|
|
// start one processor per partition
|
|
var wg sync.WaitGroup
|
|
semaphore := make(chan struct{}, sub.SubscriberConfig.MaxPartitionCount)
|
|
|
|
for message := range sub.brokerPartitionAssignmentChan {
|
|
if assigned := message.GetAssignment(); assigned != nil {
|
|
wg.Add(1)
|
|
semaphore <- struct{}{}
|
|
|
|
topicPartition := topic.FromPbPartition(assigned.PartitionAssignment.Partition)
|
|
|
|
// wait until no covering partition is still in progress
|
|
sub.waitUntilNoOverlappingPartitionInFlight(topicPartition)
|
|
|
|
// start a processors
|
|
stopChan := make(chan struct{})
|
|
sub.activeProcessorsLock.Lock()
|
|
sub.activeProcessors[topicPartition] = &ProcessorState{
|
|
stopCh: stopChan,
|
|
}
|
|
sub.activeProcessorsLock.Unlock()
|
|
|
|
go func(assigned *mq_pb.BrokerPartitionAssignment, topicPartition topic.Partition) {
|
|
defer func() {
|
|
sub.activeProcessorsLock.Lock()
|
|
delete(sub.activeProcessors, topicPartition)
|
|
sub.activeProcessorsLock.Unlock()
|
|
|
|
<-semaphore
|
|
wg.Done()
|
|
}()
|
|
glog.V(0).Infof("subscriber %s/%s assigned partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
|
|
sub.brokerPartitionAssignmentAckChan <- &mq_pb.SubscriberToSubCoordinatorRequest{
|
|
Message: &mq_pb.SubscriberToSubCoordinatorRequest_AckAssignment{
|
|
AckAssignment: &mq_pb.SubscriberToSubCoordinatorRequest_AckAssignmentMessage{
|
|
Partition: assigned.Partition,
|
|
},
|
|
},
|
|
}
|
|
|
|
executors := util.NewLimitedConcurrentExecutor(int(sub.SubscriberConfig.SlidingWindowSize))
|
|
onDataMessageFn := func(m *mq_pb.SubscribeMessageResponse_Data) {
|
|
executors.Execute(func() {
|
|
if sub.OnDataMessageFunc != nil {
|
|
sub.OnDataMessageFunc(m)
|
|
}
|
|
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 {
|
|
glog.V(0).Infof("subscriber %s/%s partition %+v at %v completed", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, assigned.Partition, assigned.LeaderBroker)
|
|
}
|
|
sub.brokerPartitionAssignmentAckChan <- &mq_pb.SubscriberToSubCoordinatorRequest{
|
|
Message: &mq_pb.SubscriberToSubCoordinatorRequest_AckUnAssignment{
|
|
AckUnAssignment: &mq_pb.SubscriberToSubCoordinatorRequest_AckUnAssignmentMessage{
|
|
Partition: assigned.Partition,
|
|
},
|
|
},
|
|
}
|
|
}(assigned.PartitionAssignment, topicPartition)
|
|
}
|
|
if unAssignment := message.GetUnAssignment(); unAssignment != nil {
|
|
topicPartition := topic.FromPbPartition(unAssignment.Partition)
|
|
sub.activeProcessorsLock.Lock()
|
|
if processor, found := sub.activeProcessors[topicPartition]; found {
|
|
close(processor.stopCh)
|
|
delete(sub.activeProcessors, topicPartition)
|
|
}
|
|
sub.activeProcessorsLock.Unlock()
|
|
}
|
|
}
|
|
|
|
wg.Wait()
|
|
|
|
}
|
|
|
|
func (sub *TopicSubscriber) waitUntilNoOverlappingPartitionInFlight(topicPartition topic.Partition) {
|
|
foundOverlapping := true
|
|
for foundOverlapping {
|
|
sub.activeProcessorsLock.Lock()
|
|
foundOverlapping = false
|
|
var overlappedPartition topic.Partition
|
|
for partition, _ := range sub.activeProcessors {
|
|
if partition.Overlaps(topicPartition) {
|
|
if partition.Equals(topicPartition) {
|
|
continue
|
|
}
|
|
foundOverlapping = true
|
|
overlappedPartition = partition
|
|
break
|
|
}
|
|
}
|
|
sub.activeProcessorsLock.Unlock()
|
|
if foundOverlapping {
|
|
glog.V(0).Infof("subscriber %s new partition %v waiting for partition %+v to complete", sub.ContentConfig.Topic, topicPartition, overlappedPartition)
|
|
time.Sleep(1 * time.Second)
|
|
}
|
|
}
|
|
}
|