mirror of
https://gitee.com/milvus-io/milvus.git
synced 2024-12-02 03:48:37 +08:00
fix: rocksmq consume too slow when the channel is full (#36617)
issue: #36569 Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
parent
4f50b7dbab
commit
addced27f9
@ -15,6 +15,7 @@ import (
|
||||
"reflect"
|
||||
"sync"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
@ -22,6 +23,10 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/mq/mqimpl/rocksmq/server"
|
||||
)
|
||||
|
||||
const (
|
||||
minimalConsumePendingBufferSize = 16
|
||||
)
|
||||
|
||||
type client struct {
|
||||
server RocksMQ
|
||||
wg *sync.WaitGroup
|
||||
@ -118,43 +123,72 @@ func (c *client) Subscribe(options ConsumerOptions) (Consumer, error) {
|
||||
|
||||
func (c *client) consume(consumer *consumer) {
|
||||
defer c.wg.Done()
|
||||
|
||||
if err := c.blockUntilInitDone(consumer); err != nil {
|
||||
log.Warn("consumer init failed", zap.Error(err))
|
||||
return
|
||||
}
|
||||
|
||||
var pendingMsgs []*RmqMessage
|
||||
for {
|
||||
if len(pendingMsgs) == 0 {
|
||||
pendingMsgs = c.tryToConsume(consumer)
|
||||
}
|
||||
|
||||
var consumerCh chan<- common.Message
|
||||
var waitForSent *RmqMessage
|
||||
var newIncomingMsgCh <-chan struct{}
|
||||
if len(pendingMsgs) > 0 {
|
||||
// If there's pending sent messages, we can try to deliver them first.
|
||||
consumerCh = consumer.messageCh
|
||||
waitForSent = pendingMsgs[0]
|
||||
} else {
|
||||
// If there's no more pending messages, we can wait for new incoming messages.
|
||||
// !!! TODO: MsgMutex may lost, not sync up with the consumer,
|
||||
// so the tailing message cannot be consumed if no new producing message.
|
||||
newIncomingMsgCh = consumer.MsgMutex()
|
||||
}
|
||||
|
||||
select {
|
||||
case <-c.closeCh:
|
||||
log.Info("Client is closed, consumer goroutine exit")
|
||||
return
|
||||
case _, ok := <-consumer.initCh:
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
c.deliver(consumer)
|
||||
case _, ok := <-consumer.MsgMutex():
|
||||
case consumerCh <- waitForSent:
|
||||
pendingMsgs = pendingMsgs[1:]
|
||||
case _, ok := <-newIncomingMsgCh:
|
||||
if !ok {
|
||||
// consumer MsgMutex closed, goroutine exit
|
||||
log.Debug("Consumer MsgMutex closed")
|
||||
log.Info("Consumer MsgMutex closed")
|
||||
return
|
||||
}
|
||||
c.deliver(consumer)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *client) deliver(consumer *consumer) {
|
||||
for {
|
||||
n := cap(consumer.messageCh) - len(consumer.messageCh)
|
||||
if n == 0 {
|
||||
return
|
||||
// blockUntilInitDone block until consumer is initialized
|
||||
func (c *client) blockUntilInitDone(consumer *consumer) error {
|
||||
select {
|
||||
case <-c.closeCh:
|
||||
return errors.New("client is closed")
|
||||
case _, ok := <-consumer.initCh:
|
||||
if !ok {
|
||||
return errors.New("consumer init failure")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func (c *client) tryToConsume(consumer *consumer) []*RmqMessage {
|
||||
n := cap(consumer.messageCh) - len(consumer.messageCh)
|
||||
if n <= minimalConsumePendingBufferSize {
|
||||
n = minimalConsumePendingBufferSize
|
||||
}
|
||||
msgs, err := consumer.client.server.Consume(consumer.topic, consumer.consumerName, n)
|
||||
if err != nil {
|
||||
log.Warn("Consumer's goroutine cannot consume from (" + consumer.topic + "," + consumer.consumerName + "): " + err.Error())
|
||||
break
|
||||
}
|
||||
|
||||
// no more msgs
|
||||
if len(msgs) == 0 {
|
||||
break
|
||||
return nil
|
||||
}
|
||||
rmqMsgs := make([]*RmqMessage, 0, len(msgs))
|
||||
for _, msg := range msgs {
|
||||
// This is the hack, we put property into pl
|
||||
properties := make(map[string]string, 0)
|
||||
@ -162,18 +196,14 @@ func (c *client) deliver(consumer *consumer) {
|
||||
if err == nil && pl != nil && pl.Base != nil {
|
||||
properties = pl.Base.Properties
|
||||
}
|
||||
select {
|
||||
case consumer.messageCh <- &RmqMessage{
|
||||
rmqMsgs = append(rmqMsgs, &RmqMessage{
|
||||
msgID: msg.MsgID,
|
||||
payload: msg.Payload,
|
||||
properties: properties,
|
||||
topic: consumer.Topic(),
|
||||
}:
|
||||
case <-c.closeCh:
|
||||
return
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
return rmqMsgs
|
||||
}
|
||||
|
||||
// Close close the channel to notify rocksmq to stop operation and close rocksmq server
|
||||
|
Loading…
Reference in New Issue
Block a user