aboutsummaryrefslogtreecommitdiff
path: root/weed/mq/broker
diff options
context:
space:
mode:
Diffstat (limited to 'weed/mq/broker')
-rw-r--r--weed/mq/broker/broker_grpc_assign.go9
-rw-r--r--weed/mq/broker/broker_grpc_pub.go135
-rw-r--r--weed/mq/broker/broker_grpc_pub_follow.go199
-rw-r--r--weed/mq/broker/broker_grpc_sub.go212
-rw-r--r--weed/mq/broker/broker_grpc_sub_coordinator.go9
-rw-r--r--weed/mq/broker/broker_topic_conf_read_write.go31
-rw-r--r--weed/mq/broker/broker_topic_partition_read_write.go10
7 files changed, 246 insertions, 359 deletions
diff --git a/weed/mq/broker/broker_grpc_assign.go b/weed/mq/broker/broker_grpc_assign.go
index 264565b7b..ee69db30d 100644
--- a/weed/mq/broker/broker_grpc_assign.go
+++ b/weed/mq/broker/broker_grpc_assign.go
@@ -14,7 +14,6 @@ import (
// AssignTopicPartitions Runs on the assigned broker, to execute the topic partition assignment
func (b *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *mq_pb.AssignTopicPartitionsRequest) (*mq_pb.AssignTopicPartitionsResponse, error) {
ret := &mq_pb.AssignTopicPartitionsResponse{}
- self := pb.ServerAddress(fmt.Sprintf("%s:%d", b.option.Ip, b.option.Port))
// drain existing topic partition subscriptions
for _, assignment := range request.BrokerPartitionAssignments {
@@ -23,12 +22,12 @@ func (b *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *m
b.accessLock.Lock()
if request.IsDraining {
// TODO drain existing topic partition subscriptions
- b.localTopicManager.RemoveTopicPartition(t, partition)
+ b.localTopicManager.RemoveLocalPartition(t, partition)
} else {
var localPartition *topic.LocalPartition
- if localPartition = b.localTopicManager.GetTopicPartition(t, partition); localPartition == nil {
- localPartition = topic.FromPbBrokerPartitionAssignment(self, partition, assignment, b.genLogFlushFunc(t, assignment.Partition), b.genLogOnDiskReadFunc(t, assignment.Partition))
- b.localTopicManager.AddTopicPartition(t, localPartition)
+ if localPartition = b.localTopicManager.GetLocalPartition(t, partition); localPartition == nil {
+ localPartition = topic.NewLocalPartition(partition, b.genLogFlushFunc(t, assignment.Partition), b.genLogOnDiskReadFunc(t, assignment.Partition))
+ b.localTopicManager.AddLocalPartition(t, localPartition)
}
}
b.accessLock.Unlock()
diff --git a/weed/mq/broker/broker_grpc_pub.go b/weed/mq/broker/broker_grpc_pub.go
index 3b68db1af..a217489de 100644
--- a/weed/mq/broker/broker_grpc_pub.go
+++ b/weed/mq/broker/broker_grpc_pub.go
@@ -5,13 +5,13 @@ import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
- "github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc/peer"
"io"
"math/rand"
"net"
"sync/atomic"
+ "time"
)
// PUB
@@ -40,73 +40,86 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
// 2. find the topic metadata owning filer
// 3. write to the filer
- var localTopicPartition *topic.LocalPartition
req, err := stream.Recv()
if err != nil {
return err
}
response := &mq_pb.PublishMessageResponse{}
// TODO check whether current broker should be the leader for the topic partition
- ackInterval := 1
initMessage := req.GetInit()
- var t topic.Topic
- var p topic.Partition
- if initMessage != nil {
- t, p = topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.Partition)
- localTopicPartition, _, err = b.GetOrGenLocalPartition(t, p)
- if err != nil {
- response.Error = fmt.Sprintf("topic %v partition %v not setup", initMessage.Topic, initMessage.Partition)
- glog.Errorf("topic %v partition %v not setup", initMessage.Topic, initMessage.Partition)
- return stream.Send(response)
- }
- ackInterval = int(initMessage.AckInterval)
- for _, follower := range initMessage.FollowerBrokers {
- followErr := b.withBrokerClient(false, pb.ServerAddress(follower), func(client mq_pb.SeaweedMessagingClient) error {
- _, err := client.PublishFollowMe(context.Background(), &mq_pb.PublishFollowMeRequest{
- Topic: initMessage.Topic,
- Partition: initMessage.Partition,
- BrokerSelf: string(b.option.BrokerAddress()),
- })
- return err
- })
- if followErr != nil {
- response.Error = fmt.Sprintf("follower %v failed: %v", follower, followErr)
- glog.Errorf("follower %v failed: %v", follower, followErr)
- return stream.Send(response)
- }
- }
- stream.Send(response)
- } else {
+ if initMessage == nil {
response.Error = fmt.Sprintf("missing init message")
glog.Errorf("missing init message")
return stream.Send(response)
}
+ // get or generate a local partition
+ t, p := topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.Partition)
+ localTopicPartition, getOrGenErr := b.GetOrGenerateLocalPartition(t, p)
+ if getOrGenErr != nil {
+ response.Error = fmt.Sprintf("topic %v not found: %v", t, getOrGenErr)
+ glog.Errorf("topic %v not found: %v", t, getOrGenErr)
+ return stream.Send(response)
+ }
+
+ // connect to follower brokers
+ if followerErr := localTopicPartition.MaybeConnectToFollowers(initMessage, b.grpcDialOption); followerErr != nil {
+ response.Error = followerErr.Error()
+ glog.Errorf("MaybeConnectToFollowers: %v", followerErr)
+ return stream.Send(response)
+ }
+
+ var receivedSequence, acknowledgedSequence int64
+ var isClosed bool
+
+ // start sending ack to publisher
+ ackInterval := int64(1)
+ if initMessage.AckInterval > 0 {
+ ackInterval = int64(initMessage.AckInterval)
+ }
+ go func() {
+ defer func() {
+ // println("stop sending ack to publisher", initMessage.PublisherName)
+ }()
+
+ lastAckTime := time.Now()
+ for !isClosed {
+ receivedSequence = atomic.LoadInt64(&localTopicPartition.AckTsNs)
+ if acknowledgedSequence < receivedSequence && (receivedSequence - acknowledgedSequence >= ackInterval || time.Since(lastAckTime) > 1*time.Second){
+ acknowledgedSequence = receivedSequence
+ response := &mq_pb.PublishMessageResponse{
+ AckSequence: acknowledgedSequence,
+ }
+ if err := stream.Send(response); err != nil {
+ glog.Errorf("Error sending response %v: %v", response, err)
+ }
+ // println("sent ack", acknowledgedSequence, "=>", initMessage.PublisherName)
+ lastAckTime = time.Now()
+ } else {
+ time.Sleep(1 * time.Second)
+ }
+ }
+ }()
+
+
+ // process each published messages
clientName := fmt.Sprintf("%v-%4d/%s/%v", findClientAddress(stream.Context()), rand.Intn(10000), initMessage.Topic, initMessage.Partition)
localTopicPartition.Publishers.AddPublisher(clientName, topic.NewLocalPublisher())
- ackCounter := 0
- var ackSequence int64
- var isStopping int32
- respChan := make(chan *mq_pb.PublishMessageResponse, 128)
defer func() {
- atomic.StoreInt32(&isStopping, 1)
- respChan <- &mq_pb.PublishMessageResponse{
- AckSequence: ackSequence,
- }
- close(respChan)
+ // remove the publisher
localTopicPartition.Publishers.RemovePublisher(clientName)
- glog.V(0).Infof("topic %v partition %v published %d messges Publisher:%d Subscriber:%d", initMessage.Topic, initMessage.Partition, ackSequence, localTopicPartition.Publishers.Size(), localTopicPartition.Subscribers.Size())
if localTopicPartition.MaybeShutdownLocalPartition() {
- b.localTopicManager.RemoveTopicPartition(t, p)
+ b.localTopicManager.RemoveLocalPartition(t, p)
+ glog.V(0).Infof("Removed local topic %v partition %v", initMessage.Topic, initMessage.Partition)
}
}()
- go func() {
- for resp := range respChan {
- if err := stream.Send(resp); err != nil {
- glog.Errorf("Error sending response %v: %v", resp, err)
- }
- }
+
+ // send a hello message
+ stream.Send(&mq_pb.PublishMessageResponse{})
+
+ defer func() {
+ isClosed = true
}()
// process each published messages
@@ -117,28 +130,26 @@ func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_Publis
if err == io.EOF {
break
}
- glog.V(0).Infof("topic %v partition %v publish stream error: %v", initMessage.Topic, initMessage.Partition, err)
- return err
+ glog.V(0).Infof("topic %v partition %v publish stream from %s error: %v", initMessage.Topic, initMessage.Partition, initMessage.PublisherName, err)
+ break
}
// Process the received message
- if dataMessage := req.GetData(); dataMessage != nil {
- localTopicPartition.Publish(dataMessage)
+ dataMessage := req.GetData()
+ if dataMessage == nil {
+ continue
}
- ackCounter++
- ackSequence++
- if ackCounter >= ackInterval {
- ackCounter = 0
- // send back the ack
- response := &mq_pb.PublishMessageResponse{
- AckSequence: ackSequence,
- }
- respChan <- response
+ // The control message should still be sent to the follower
+ // to avoid timing issue when ack messages.
+
+ // send to the local partition
+ if err = localTopicPartition.Publish(dataMessage); err != nil {
+ return fmt.Errorf("topic %v partition %v publish error: %v", initMessage.Topic, initMessage.Partition, err)
}
}
- glog.V(0).Infof("topic %v partition %v publish stream closed.", initMessage.Topic, initMessage.Partition)
+ glog.V(0).Infof("topic %v partition %v publish stream from %s closed.", initMessage.Topic, initMessage.Partition, initMessage.PublisherName)
return nil
}
diff --git a/weed/mq/broker/broker_grpc_pub_follow.go b/weed/mq/broker/broker_grpc_pub_follow.go
index 8ef85110a..d8100f021 100644
--- a/weed/mq/broker/broker_grpc_pub_follow.go
+++ b/weed/mq/broker/broker_grpc_pub_follow.go
@@ -1,96 +1,151 @@
package broker
import (
- "context"
"fmt"
+ "github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/glog"
- "github.com/seaweedfs/seaweedfs/weed/pb"
+ "github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
+ "github.com/seaweedfs/seaweedfs/weed/util/buffered_queue"
+ "github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
"io"
- "math/rand"
- "sync"
"time"
)
-func (b *MessageQueueBroker) PublishFollowMe(c context.Context, request *mq_pb.PublishFollowMeRequest) (*mq_pb.PublishFollowMeResponse, error) {
- glog.V(0).Infof("PublishFollowMe %v", request)
- var wg sync.WaitGroup
- wg.Add(1)
- var ret error
- go b.withBrokerClient(true, pb.ServerAddress(request.BrokerSelf), func(client mq_pb.SeaweedMessagingClient) error {
- followerId := rand.Int31()
- subscribeClient, err := client.FollowInMemoryMessages(context.Background(), &mq_pb.FollowInMemoryMessagesRequest{
- Message: &mq_pb.FollowInMemoryMessagesRequest_Init{
- Init: &mq_pb.FollowInMemoryMessagesRequest_InitMessage{
- ConsumerGroup: string(b.option.BrokerAddress()),
- ConsumerId: fmt.Sprintf("followMe-%d", followerId),
- FollowerId: followerId,
- Topic: request.Topic,
- PartitionOffset: &mq_pb.PartitionOffset{
- Partition: request.Partition,
- StartTsNs: 0,
- StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
- },
- },
- },
- })
+type memBuffer struct {
+ buf []byte
+ startTime time.Time
+ stopTime time.Time
+}
+func (b *MessageQueueBroker) PublishFollowMe(stream mq_pb.SeaweedMessaging_PublishFollowMeServer) (err error) {
+ var req *mq_pb.PublishFollowMeRequest
+ req, err = stream.Recv()
+ if err != nil {
+ return err
+ }
+ initMessage := req.GetInit()
+ if initMessage == nil {
+ return fmt.Errorf("missing init message")
+ }
- if err != nil {
- glog.Errorf("FollowInMemoryMessages error: %v", err)
- ret = err
- return err
- }
+ // create an in-memory queue of buffered messages
+ inMemoryBuffers := buffered_queue.NewBufferedQueue[memBuffer](4)
+ logBuffer := b.buildFollowerLogBuffer(inMemoryBuffers)
- // receive first hello message
- resp, err := subscribeClient.Recv()
+ lastFlushTsNs := time.Now().UnixNano()
+
+ // follow each published messages
+ for {
+ // receive a message
+ req, err = stream.Recv()
if err != nil {
- return fmt.Errorf("FollowInMemoryMessages recv first message error: %v", err)
- }
- if resp == nil {
- glog.V(0).Infof("doFollowInMemoryMessage recv first message nil response")
- return io.ErrUnexpectedEOF
+ if err == io.EOF {
+ err = nil
+ break
+ }
+ glog.V(0).Infof("topic %v partition %v publish stream error: %v", initMessage.Topic, initMessage.Partition, err)
+ break
}
- wg.Done()
- b.doFollowInMemoryMessage(context.Background(), subscribeClient)
+ // Process the received message
+ if dataMessage := req.GetData(); dataMessage != nil {
- return nil
- })
- wg.Wait()
- return &mq_pb.PublishFollowMeResponse{}, ret
-}
+ // TODO: change this to DataMessage
+ // log the message
+ logBuffer.AddToBuffer(dataMessage)
-func (b *MessageQueueBroker) doFollowInMemoryMessage(c context.Context, client mq_pb.SeaweedMessaging_FollowInMemoryMessagesClient) {
- for {
- resp, err := client.Recv()
- if err != nil {
- if err != io.EOF {
- glog.V(0).Infof("doFollowInMemoryMessage error: %v", err)
+ // send back the ack
+ if err := stream.Send(&mq_pb.PublishFollowMeResponse{
+ AckTsNs: dataMessage.TsNs,
+ }); err != nil {
+ glog.Errorf("Error sending response %v: %v", dataMessage, err)
+ }
+ // println("ack", string(dataMessage.Key), dataMessage.TsNs)
+ } else if closeMessage := req.GetClose(); closeMessage != nil {
+ glog.V(0).Infof("topic %v partition %v publish stream closed: %v", initMessage.Topic, initMessage.Partition, closeMessage)
+ break
+ } else if flushMessage := req.GetFlush(); flushMessage != nil {
+ glog.V(0).Infof("topic %v partition %v publish stream flushed: %v", initMessage.Topic, initMessage.Partition, flushMessage)
+
+ lastFlushTsNs = flushMessage.TsNs
+
+ // drop already flushed messages
+ for mem, found := inMemoryBuffers.PeekHead(); found; mem, found = inMemoryBuffers.PeekHead() {
+ if mem.stopTime.UnixNano() <= flushMessage.TsNs {
+ inMemoryBuffers.Dequeue()
+ // println("dropping flushed messages: ", mem.startTime.UnixNano(), mem.stopTime.UnixNano(), len(mem.buf))
+ } else {
+ break
+ }
}
- return
+
+ } else {
+ glog.Errorf("unknown message: %v", req)
}
- if resp == nil {
- glog.V(0).Infof("doFollowInMemoryMessage nil response")
- return
+ }
+
+
+ t, p := topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.Partition)
+
+ logBuffer.ShutdownLogBuffer()
+ // wait until all messages are sent to inMemoryBuffers
+ for !logBuffer.IsAllFlushed() {
+ time.Sleep(113 * time.Millisecond)
+ }
+
+ topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
+ partitionGeneration := time.Unix(0, p.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
+ partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, p.RangeStart, p.RangeStop)
+
+
+ // flush the remaining messages
+ inMemoryBuffers.CloseInput()
+ for mem, found := inMemoryBuffers.Dequeue(); found; mem, found = inMemoryBuffers.Dequeue() {
+ if len(mem.buf) == 0 {
+ continue
}
- if resp.Message != nil {
- // process ctrl message or data message
- switch m := resp.Message.(type) {
- case *mq_pb.FollowInMemoryMessagesResponse_Data:
- // process data message
- print("d")
- case *mq_pb.FollowInMemoryMessagesResponse_Ctrl:
- // process ctrl message
- if m.Ctrl.FlushedSequence > 0 {
- flushTime := time.Unix(0, m.Ctrl.FlushedSequence)
- glog.V(0).Infof("doFollowInMemoryMessage flushTime: %v", flushTime)
- }
- if m.Ctrl.FollowerChangedToId != 0 {
- // follower changed
- glog.V(0).Infof("doFollowInMemoryMessage follower changed to %d", m.Ctrl.FollowerChangedToId)
- return
- }
+
+ startTime, stopTime := mem.startTime.UTC(), mem.stopTime.UTC()
+
+ if stopTime.UnixNano() <= lastFlushTsNs {
+ glog.V(0).Infof("dropping remaining data at %v %v", t, p)
+ continue
+ }
+
+ // TODO trim data earlier than lastFlushTsNs
+
+ targetFile := fmt.Sprintf("%s/%s", partitionDir, startTime.Format(topic.TIME_FORMAT))
+
+ for {
+ if err := b.appendToFile(targetFile, mem.buf); err != nil {
+ glog.V(0).Infof("metadata log write failed %s: %v", targetFile, err)
+ time.Sleep(737 * time.Millisecond)
+ } else {
+ break
}
}
+
+ glog.V(0).Infof("flushed remaining data at %v to %s size %d", mem.stopTime.UnixNano(), targetFile, len(mem.buf))
}
+
+ glog.V(0).Infof("shut down follower for %v %v", t, p)
+
+ return err
+}
+
+func (b *MessageQueueBroker) buildFollowerLogBuffer(inMemoryBuffers *buffered_queue.BufferedQueue[memBuffer]) *log_buffer.LogBuffer {
+ lb := log_buffer.NewLogBuffer("follower",
+ 2*time.Minute, func(logBuffer *log_buffer.LogBuffer, startTime, stopTime time.Time, buf []byte) {
+ if len(buf) == 0 {
+ return
+ }
+ inMemoryBuffers.Enqueue(memBuffer{
+ buf: buf,
+ startTime: startTime,
+ stopTime: stopTime,
+ })
+ glog.V(0).Infof("queue up %d~%d size %d", startTime.UnixNano(), stopTime.UnixNano(), len(buf))
+ }, nil, func() {
+ })
+ return lb
}
diff --git a/weed/mq/broker/broker_grpc_sub.go b/weed/mq/broker/broker_grpc_sub.go
index 1141ff47f..02488b2b0 100644
--- a/weed/mq/broker/broker_grpc_sub.go
+++ b/weed/mq/broker/broker_grpc_sub.go
@@ -8,7 +8,6 @@ import (
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
- "sync/atomic"
"time"
)
@@ -17,40 +16,20 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest
ctx := stream.Context()
clientName := fmt.Sprintf("%s/%s-%s", req.GetInit().ConsumerGroup, req.GetInit().ConsumerId, req.GetInit().ClientId)
+ initMessage := req.GetInit()
+ if initMessage == nil {
+ glog.Errorf("missing init message")
+ return fmt.Errorf("missing init message")
+ }
+
t := topic.FromPbTopic(req.GetInit().Topic)
partition := topic.FromPbPartition(req.GetInit().GetPartitionOffset().GetPartition())
glog.V(0).Infof("Subscriber %s on %v %v connected", req.GetInit().ConsumerId, t, partition)
- waitIntervalCount := 0
-
- var localTopicPartition *topic.LocalPartition
- for localTopicPartition == nil {
- localTopicPartition, _, err = b.GetOrGenLocalPartition(t, partition)
- if err != nil {
- glog.V(1).Infof("topic %v partition %v not setup", t, partition)
- }
- if localTopicPartition != nil {
- break
- }
- waitIntervalCount++
- if waitIntervalCount > 10 {
- waitIntervalCount = 10
- }
- time.Sleep(time.Duration(waitIntervalCount) * 337 * time.Millisecond)
- // Check if the client has disconnected by monitoring the context
- select {
- case <-ctx.Done():
- err := ctx.Err()
- if err == context.Canceled {
- // Client disconnected
- return nil
- }
- glog.V(0).Infof("Subscriber %s disconnected: %v", clientName, err)
- return nil
- default:
- // Continue processing the request
- }
+ localTopicPartition, getOrGenErr := b.GetOrGenerateLocalPartition(t, partition)
+ if getOrGenErr != nil {
+ return getOrGenErr
}
localTopicPartition.Subscribers.AddSubscriber(clientName, topic.NewLocalSubscriber())
@@ -64,7 +43,7 @@ func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest
localTopicPartition.Subscribers.RemoveSubscriber(clientName)
glog.V(0).Infof("Subscriber %s on %v %v disconnected, sent %d", clientName, t, partition, counter)
if localTopicPartition.MaybeShutdownLocalPartition() {
- b.localTopicManager.RemoveTopicPartition(t, partition)
+ b.localTopicManager.RemoveLocalPartition(t, partition)
}
}()
@@ -129,174 +108,3 @@ func getRequestPosition(offset *mq_pb.PartitionOffset) (startPosition log_buffer
}
return
}
-
-func (b *MessageQueueBroker) FollowInMemoryMessages(req *mq_pb.FollowInMemoryMessagesRequest, stream mq_pb.SeaweedMessaging_FollowInMemoryMessagesServer) (err error) {
- ctx := stream.Context()
- clientName := req.GetInit().ConsumerId
-
- t := topic.FromPbTopic(req.GetInit().Topic)
- partition := topic.FromPbPartition(req.GetInit().GetPartitionOffset().GetPartition())
-
- glog.V(0).Infof("FollowInMemoryMessages %s on %v %v connected", clientName, t, partition)
-
- waitIntervalCount := 0
-
- var localTopicPartition *topic.LocalPartition
- for localTopicPartition == nil {
- localTopicPartition, _, err = b.GetOrGenLocalPartition(t, partition)
- if err != nil {
- glog.V(1).Infof("topic %v partition %v not setup", t, partition)
- }
- if localTopicPartition != nil {
- break
- }
- waitIntervalCount++
- if waitIntervalCount > 32 {
- waitIntervalCount = 32
- }
- time.Sleep(time.Duration(waitIntervalCount) * 137 * time.Millisecond)
- // Check if the client has disconnected by monitoring the context
- select {
- case <-ctx.Done():
- err := ctx.Err()
- if err == context.Canceled {
- // Client disconnected
- return nil
- }
- glog.V(0).Infof("FollowInMemoryMessages %s disconnected: %v", clientName, err)
- return nil
- default:
- // Continue processing the request
- }
- }
-
- // set the current follower id
- followerId := req.GetInit().FollowerId
- atomic.StoreInt32(&localTopicPartition.FollowerId, followerId)
-
- glog.V(0).Infof("FollowInMemoryMessages %s connected on %v %v", clientName, t, partition)
- isConnected := true
- sleepIntervalCount := 0
-
- var counter int64
- defer func() {
- isConnected = false
- glog.V(0).Infof("FollowInMemoryMessages %s on %v %v disconnected, sent %d", clientName, t, partition, counter)
- }()
-
- // send first hello message
- // to indicate the follower is connected
- stream.Send(&mq_pb.FollowInMemoryMessagesResponse{
- Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{
- Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{},
- },
- })
-
- var startPosition log_buffer.MessagePosition
- if req.GetInit() != nil && req.GetInit().GetPartitionOffset() != nil {
- startPosition = getRequestPosition(req.GetInit().GetPartitionOffset())
- }
-
- var prevFlushTsNs int64
-
- _, _, err = localTopicPartition.LogBuffer.LoopProcessLogData(clientName, startPosition, 0, func() bool {
- if !isConnected {
- return false
- }
- sleepIntervalCount++
- if sleepIntervalCount > 32 {
- sleepIntervalCount = 32
- }
- time.Sleep(time.Duration(sleepIntervalCount) * 137 * time.Millisecond)
-
- if localTopicPartition.LogBuffer.IsStopping() {
- newFollowerId := atomic.LoadInt32(&localTopicPartition.FollowerId)
- glog.V(0).Infof("FollowInMemoryMessages1 %s on %v %v follower id changed to %d", clientName, t, partition, newFollowerId)
- stream.Send(&mq_pb.FollowInMemoryMessagesResponse{
- Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{
- Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{
- FollowerChangedToId: newFollowerId,
- },
- },
- })
- return false
- }
-
- // Check if the client has disconnected by monitoring the context
- select {
- case <-ctx.Done():
- err := ctx.Err()
- if err == context.Canceled {
- // Client disconnected
- return false
- }
- glog.V(0).Infof("Subscriber %s disconnected: %v", clientName, err)
- return false
- default:
- // Continue processing the request
- }
-
- // send the last flushed sequence
- flushTsNs := atomic.LoadInt64(&localTopicPartition.LogBuffer.LastFlushTsNs)
- if flushTsNs != prevFlushTsNs {
- prevFlushTsNs = flushTsNs
- stream.Send(&mq_pb.FollowInMemoryMessagesResponse{
- Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{
- Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{
- FlushedSequence: flushTsNs,
- },
- },
- })
- }
-
- return true
- }, func(logEntry *filer_pb.LogEntry) (bool, error) {
- // reset the sleep interval count
- sleepIntervalCount = 0
-
- // check the follower id
- newFollowerId := atomic.LoadInt32(&localTopicPartition.FollowerId)
- if newFollowerId != followerId {
- glog.V(0).Infof("FollowInMemoryMessages2 %s on %v %v follower id %d changed to %d", clientName, t, partition, followerId, newFollowerId)
- stream.Send(&mq_pb.FollowInMemoryMessagesResponse{
- Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{
- Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{
- FollowerChangedToId: newFollowerId,
- },
- },
- })
- return true, nil
- }
-
- // send the last flushed sequence
- flushTsNs := atomic.LoadInt64(&localTopicPartition.LogBuffer.LastFlushTsNs)
- if flushTsNs != prevFlushTsNs {
- prevFlushTsNs = flushTsNs
- stream.Send(&mq_pb.FollowInMemoryMessagesResponse{
- Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{
- Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{
- FlushedSequence: flushTsNs,
- },
- },
- })
- }
-
- // send the log entry
- if err := stream.Send(&mq_pb.FollowInMemoryMessagesResponse{
- Message: &mq_pb.FollowInMemoryMessagesResponse_Data{
- Data: &mq_pb.DataMessage{
- Key: logEntry.Key,
- Value: logEntry.Data,
- TsNs: logEntry.TsNs,
- },
- }}); err != nil {
- glog.Errorf("Error sending setup response: %v", err)
- return false, err
- }
-
- counter++
- return false, nil
- })
-
- return err
-}
diff --git a/weed/mq/broker/broker_grpc_sub_coordinator.go b/weed/mq/broker/broker_grpc_sub_coordinator.go
index 89c221af5..3fd97f1c2 100644
--- a/weed/mq/broker/broker_grpc_sub_coordinator.go
+++ b/weed/mq/broker/broker_grpc_sub_coordinator.go
@@ -39,18 +39,11 @@ func (b *MessageQueueBroker) SubscriberToSubCoordinator(stream mq_pb.SeaweedMess
go func() {
// try to load the partition assignment from filer
if conf, err := b.readTopicConfFromFiler(topic.FromPbTopic(initMessage.Topic)); err == nil {
- assignedPartitions := make([]*mq_pb.SubscriberToSubCoordinatorResponse_AssignedPartition, len(conf.BrokerPartitionAssignments))
- for i, assignment := range conf.BrokerPartitionAssignments {
- assignedPartitions[i] = &mq_pb.SubscriberToSubCoordinatorResponse_AssignedPartition{
- Partition: assignment.Partition,
- Broker: assignment.LeaderBroker,
- }
- }
// send partition assignment to subscriber
cgi.ResponseChan <- &mq_pb.SubscriberToSubCoordinatorResponse{
Message: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment_{
Assignment: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment{
- AssignedPartitions: assignedPartitions,
+ PartitionAssignments: conf.BrokerPartitionAssignments,
},
},
}
diff --git a/weed/mq/broker/broker_topic_conf_read_write.go b/weed/mq/broker/broker_topic_conf_read_write.go
index 35d95c0e4..cddd6cf1c 100644
--- a/weed/mq/broker/broker_topic_conf_read_write.go
+++ b/weed/mq/broker/broker_topic_conf_read_write.go
@@ -56,12 +56,27 @@ func (b *MessageQueueBroker) readTopicConfFromFiler(t topic.Topic) (conf *mq_pb.
return conf, nil
}
-func (b *MessageQueueBroker) GetOrGenLocalPartition(t topic.Topic, partition topic.Partition) (localPartition *topic.LocalPartition, isGenerated bool, err error) {
+func (b *MessageQueueBroker) GetOrGenerateLocalPartition(t topic.Topic, partition topic.Partition) (localTopicPartition *topic.LocalPartition, getOrGenError error) {
+ // get or generate a local partition
+ conf, readConfErr := b.readTopicConfFromFiler(t)
+ if readConfErr != nil {
+ glog.Errorf("topic %v not found: %v", t, readConfErr)
+ return nil, fmt.Errorf("topic %v not found: %v", t, readConfErr)
+ }
+ localTopicPartition, _, getOrGenError = b.doGetOrGenLocalPartition(t, partition, conf)
+ if getOrGenError != nil {
+ glog.Errorf("topic %v partition %v not setup: %v", t, partition, getOrGenError)
+ return nil, fmt.Errorf("topic %v partition %v not setup: %v", t, partition, getOrGenError)
+ }
+ return localTopicPartition, nil
+}
+
+func (b *MessageQueueBroker) doGetOrGenLocalPartition(t topic.Topic, partition topic.Partition, conf *mq_pb.ConfigureTopicResponse) (localPartition *topic.LocalPartition, isGenerated bool, err error) {
b.accessLock.Lock()
defer b.accessLock.Unlock()
- if localPartition = b.localTopicManager.GetTopicPartition(t, partition); localPartition == nil {
- localPartition, isGenerated, err = b.genLocalPartitionFromFiler(t, partition)
+ if localPartition = b.localTopicManager.GetLocalPartition(t, partition); localPartition == nil {
+ localPartition, isGenerated, err = b.genLocalPartitionFromFiler(t, partition, conf)
if err != nil {
return nil, false, err
}
@@ -69,16 +84,12 @@ func (b *MessageQueueBroker) GetOrGenLocalPartition(t topic.Topic, partition top
return localPartition, isGenerated, nil
}
-func (b *MessageQueueBroker) genLocalPartitionFromFiler(t topic.Topic, partition topic.Partition) (localPartition *topic.LocalPartition, isGenerated bool, err error) {
+func (b *MessageQueueBroker) genLocalPartitionFromFiler(t topic.Topic, partition topic.Partition, conf *mq_pb.ConfigureTopicResponse) (localPartition *topic.LocalPartition, isGenerated bool, err error) {
self := b.option.BrokerAddress()
- conf, err := b.readTopicConfFromFiler(t)
- if err != nil {
- return nil, isGenerated, err
- }
for _, assignment := range conf.BrokerPartitionAssignments {
if assignment.LeaderBroker == string(self) && partition.Equals(topic.FromPbPartition(assignment.Partition)) {
- localPartition = topic.FromPbBrokerPartitionAssignment(b.option.BrokerAddress(), partition, assignment, b.genLogFlushFunc(t, assignment.Partition), b.genLogOnDiskReadFunc(t, assignment.Partition))
- b.localTopicManager.AddTopicPartition(t, localPartition)
+ localPartition = topic.NewLocalPartition(partition, b.genLogFlushFunc(t, assignment.Partition), b.genLogOnDiskReadFunc(t, assignment.Partition))
+ b.localTopicManager.AddLocalPartition(t, localPartition)
isGenerated = true
break
}
diff --git a/weed/mq/broker/broker_topic_partition_read_write.go b/weed/mq/broker/broker_topic_partition_read_write.go
index a058d8da5..50470f879 100644
--- a/weed/mq/broker/broker_topic_partition_read_write.go
+++ b/weed/mq/broker/broker_topic_partition_read_write.go
@@ -41,6 +41,16 @@ func (b *MessageQueueBroker) genLogFlushFunc(t topic.Topic, partition *mq_pb.Par
}
atomic.StoreInt64(&logBuffer.LastFlushTsNs, stopTime.UnixNano())
+
+ b.accessLock.Lock()
+ defer b.accessLock.Unlock()
+ p := topic.FromPbPartition(partition)
+ if localPartition:=b.localTopicManager.GetLocalPartition(t, p); localPartition!=nil {
+ localPartition.NotifyLogFlushed(logBuffer.LastFlushTsNs)
+ }
+
+ println("flushing at", logBuffer.LastFlushTsNs, "to", targetFile, "size", len(buf))
+
}
}