mirror of https://github.com/milvus-io/milvus.git
fix: [10kcp] Fix consume blocked due to too many consumers (#38456)
This PR limits the maximum number of consumers per pchannel to 10 for each QueryNode and DataNode. issue: https://github.com/milvus-io/milvus/issues/37630 pr: https://github.com/milvus-io/milvus/pull/38455 --------- Signed-off-by: SimFG <bang.fu@zilliz.com> Signed-off-by: bigsheeper <yihao.dai@zilliz.com> Co-authored-by: SimFG <bang.fu@zilliz.com>pull/38521/head
parent
df4d5e1096
commit
de78de7689
|
@ -170,6 +170,12 @@ mq:
|
|||
mergeCheckInterval: 1 # the interval time(in seconds) for dispatcher to check whether to merge
|
||||
targetBufSize: 16 # the lenth of channel buffer for targe
|
||||
maxTolerantLag: 3 # Default value: "3", the timeout(in seconds) that target sends msgPack
|
||||
# The maximum number of dispatchers per physical channel, primarily to limit
|
||||
# the number of MQ consumers and prevent performance issues
|
||||
# (e.g., during recovery when a large number of channels are watched).
|
||||
maxDispatcherNumPerPchannel: 10
|
||||
retrySleep: 300 # register retry sleep in seconds
|
||||
retryTimeout: 5 # register retry timeout in seconds
|
||||
|
||||
# Related configuration of pulsar, used to manage Milvus logs of recent mutation operations, output streaming log, and provide log publish-subscribe services.
|
||||
pulsar:
|
||||
|
|
|
@ -21,6 +21,7 @@ import (
|
|||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
|
@ -31,6 +32,7 @@ import (
|
|||
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/retry"
|
||||
"github.com/milvus-io/milvus/pkg/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
@ -41,24 +43,49 @@ import (
|
|||
// flowgraph ddNode.
|
||||
func newDmInputNode(initCtx context.Context, dispatcherClient msgdispatcher.Client, seekPos *msgpb.MsgPosition, dmNodeConfig *nodeConfig) (*flowgraph.InputNode, error) {
|
||||
log := log.With(zap.Int64("nodeID", paramtable.GetNodeID()),
|
||||
zap.Int64("collectionID", dmNodeConfig.collectionID),
|
||||
zap.String("vchannel", dmNodeConfig.vChannelName))
|
||||
var err error
|
||||
var input <-chan *msgstream.MsgPack
|
||||
|
||||
var (
|
||||
input <-chan *msgstream.MsgPack
|
||||
err error
|
||||
start = time.Now()
|
||||
)
|
||||
|
||||
if seekPos != nil && len(seekPos.MsgID) != 0 {
|
||||
input, err = dispatcherClient.Register(initCtx, dmNodeConfig.vChannelName, seekPos, common.SubscriptionPositionUnknown)
|
||||
err := retry.Handle(initCtx, func() (bool, error) {
|
||||
input, err = dispatcherClient.Register(initCtx, dmNodeConfig.vChannelName, seekPos, common.SubscriptionPositionUnknown)
|
||||
if err != nil {
|
||||
log.Warn("datanode consume failed", zap.Error(err))
|
||||
return errors.Is(err, msgdispatcher.ErrTooManyConsumers), err
|
||||
}
|
||||
return false, nil
|
||||
}, retry.Sleep(paramtable.Get().MQCfg.RetrySleep.GetAsDuration(time.Second)), // 5 seconds
|
||||
retry.MaxSleepTime(paramtable.Get().MQCfg.RetryTimeout.GetAsDuration(time.Second))) // 5 minutes
|
||||
if err != nil {
|
||||
log.Warn("datanode consume failed after retried", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
log.Info("datanode seek successfully when register to msgDispatcher",
|
||||
zap.ByteString("msgID", seekPos.GetMsgID()),
|
||||
zap.Time("tsTime", tsoutil.PhysicalTime(seekPos.GetTimestamp())),
|
||||
zap.Duration("tsLag", time.Since(tsoutil.PhysicalTime(seekPos.GetTimestamp()))))
|
||||
zap.Duration("tsLag", time.Since(tsoutil.PhysicalTime(seekPos.GetTimestamp()))),
|
||||
zap.Duration("dur", time.Since(start)))
|
||||
} else {
|
||||
input, err = dispatcherClient.Register(initCtx, dmNodeConfig.vChannelName, nil, common.SubscriptionPositionEarliest)
|
||||
err = retry.Handle(initCtx, func() (bool, error) {
|
||||
input, err = dispatcherClient.Register(initCtx, dmNodeConfig.vChannelName, nil, common.SubscriptionPositionUnknown)
|
||||
if err != nil {
|
||||
log.Warn("datanode consume failed", zap.Error(err))
|
||||
return errors.Is(err, msgdispatcher.ErrTooManyConsumers), err
|
||||
}
|
||||
return false, nil
|
||||
}, retry.Sleep(paramtable.Get().MQCfg.RetrySleep.GetAsDuration(time.Second)), // 5 seconds
|
||||
retry.MaxSleepTime(paramtable.Get().MQCfg.RetryTimeout.GetAsDuration(time.Second))) // 5 minutes
|
||||
if err != nil {
|
||||
log.Warn("datanode consume failed after retried", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
log.Info("datanode consume successfully when register to msgDispatcher")
|
||||
}
|
||||
|
||||
|
|
|
@ -101,7 +101,7 @@ func (suite *PipelineManagerTestSuite) TestBasic() {
|
|||
suite.NotNil(pipeline)
|
||||
|
||||
// Init Consumer
|
||||
err = pipeline.ConsumeMsgStream(&msgpb.MsgPosition{})
|
||||
err = pipeline.ConsumeMsgStream(context.TODO(), &msgpb.MsgPosition{})
|
||||
suite.NoError(err)
|
||||
|
||||
// Start pipeline
|
||||
|
|
|
@ -147,7 +147,7 @@ func (suite *PipelineTestSuite) TestBasic() {
|
|||
suite.NoError(err)
|
||||
|
||||
// Init Consumer
|
||||
err = pipeline.ConsumeMsgStream(&msgpb.MsgPosition{})
|
||||
err = pipeline.ConsumeMsgStream(context.TODO(), &msgpb.MsgPosition{})
|
||||
suite.NoError(err)
|
||||
|
||||
err = pipeline.Start()
|
||||
|
|
|
@ -324,7 +324,7 @@ func (node *QueryNode) WatchDmChannels(ctx context.Context, req *querypb.WatchDm
|
|||
MsgID: channel.SeekPosition.MsgID,
|
||||
Timestamp: channel.SeekPosition.Timestamp,
|
||||
}
|
||||
err = pipeline.ConsumeMsgStream(position)
|
||||
err = pipeline.ConsumeMsgStream(ctx, position)
|
||||
if err != nil {
|
||||
err = merr.WrapErrServiceUnavailable(err.Error(), "InitPipelineFailed")
|
||||
log.Warn(err.Error(),
|
||||
|
|
|
@ -21,6 +21,7 @@ import (
|
|||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
|
@ -28,12 +29,14 @@ import (
|
|||
"github.com/milvus-io/milvus/pkg/mq/common"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/retry"
|
||||
"github.com/milvus-io/milvus/pkg/util/tsoutil"
|
||||
)
|
||||
|
||||
type StreamPipeline interface {
|
||||
Pipeline
|
||||
ConsumeMsgStream(position *msgpb.MsgPosition) error
|
||||
ConsumeMsgStream(ctx context.Context, position *msgpb.MsgPosition) error
|
||||
}
|
||||
|
||||
type streamPipeline struct {
|
||||
|
@ -63,7 +66,7 @@ func (p *streamPipeline) work() {
|
|||
}
|
||||
}
|
||||
|
||||
func (p *streamPipeline) ConsumeMsgStream(position *msgpb.MsgPosition) error {
|
||||
func (p *streamPipeline) ConsumeMsgStream(ctx context.Context, position *msgpb.MsgPosition) error {
|
||||
var err error
|
||||
if position == nil {
|
||||
log.Error("seek stream to nil position")
|
||||
|
@ -71,11 +74,20 @@ func (p *streamPipeline) ConsumeMsgStream(position *msgpb.MsgPosition) error {
|
|||
}
|
||||
|
||||
start := time.Now()
|
||||
p.input, err = p.dispatcher.Register(context.TODO(), p.vChannel, position, common.SubscriptionPositionUnknown)
|
||||
err = retry.Handle(ctx, func() (bool, error) {
|
||||
p.input, err = p.dispatcher.Register(ctx, p.vChannel, position, common.SubscriptionPositionUnknown)
|
||||
if err != nil {
|
||||
log.Warn("dispatcher register failed", zap.String("channel", position.ChannelName), zap.Error(err))
|
||||
return errors.Is(err, msgdispatcher.ErrTooManyConsumers), err
|
||||
}
|
||||
return false, nil
|
||||
}, retry.Sleep(paramtable.Get().MQCfg.RetrySleep.GetAsDuration(time.Second)), // 5 seconds
|
||||
retry.MaxSleepTime(paramtable.Get().MQCfg.RetryTimeout.GetAsDuration(time.Second))) // 5 minutes
|
||||
if err != nil {
|
||||
log.Error("dispatcher register failed", zap.String("channel", position.ChannelName))
|
||||
log.Error("dispatcher register failed after retried", zap.String("channel", position.ChannelName), zap.Error(err))
|
||||
return WrapErrRegDispather(err)
|
||||
}
|
||||
|
||||
ts, _ := tsoutil.ParseTS(position.GetTimestamp())
|
||||
log.Info("stream pipeline seeks from position with msgDispatcher",
|
||||
zap.String("pchannel", position.ChannelName),
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package pipeline
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
|
@ -63,7 +64,7 @@ func (suite *StreamPipelineSuite) TestBasic() {
|
|||
})
|
||||
}
|
||||
|
||||
err := suite.pipeline.ConsumeMsgStream(&msgpb.MsgPosition{})
|
||||
err := suite.pipeline.ConsumeMsgStream(context.TODO(), &msgpb.MsgPosition{})
|
||||
suite.NoError(err)
|
||||
|
||||
suite.pipeline.Start()
|
||||
|
|
|
@ -18,8 +18,9 @@ package msgdispatcher
|
|||
|
||||
import (
|
||||
"context"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
|
@ -27,8 +28,13 @@ import (
|
|||
"github.com/milvus-io/milvus/pkg/mq/common"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/lock"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
var ErrTooManyConsumers = errors.New("consumer number limit exceeded")
|
||||
|
||||
type (
|
||||
Pos = msgpb.MsgPosition
|
||||
MsgPack = msgstream.MsgPack
|
||||
|
@ -46,17 +52,18 @@ var _ Client = (*client)(nil)
|
|||
type client struct {
|
||||
role string
|
||||
nodeID int64
|
||||
managers map[string]DispatcherManager
|
||||
managerMut sync.Mutex
|
||||
managers *typeutil.ConcurrentMap[string, DispatcherManager]
|
||||
managerMut *lock.KeyLock[string]
|
||||
factory msgstream.Factory
|
||||
}
|
||||
|
||||
func NewClient(factory msgstream.Factory, role string, nodeID int64) Client {
|
||||
return &client{
|
||||
role: role,
|
||||
nodeID: nodeID,
|
||||
factory: factory,
|
||||
managers: make(map[string]DispatcherManager),
|
||||
role: role,
|
||||
nodeID: nodeID,
|
||||
factory: factory,
|
||||
managers: typeutil.NewConcurrentMap[string, DispatcherManager](),
|
||||
managerMut: lock.NewKeyLock[string](),
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -64,52 +71,61 @@ func (c *client) Register(ctx context.Context, vchannel string, pos *Pos, subPos
|
|||
log := log.With(zap.String("role", c.role),
|
||||
zap.Int64("nodeID", c.nodeID), zap.String("vchannel", vchannel))
|
||||
pchannel := funcutil.ToPhysicalChannel(vchannel)
|
||||
c.managerMut.Lock()
|
||||
defer c.managerMut.Unlock()
|
||||
start := time.Now()
|
||||
c.managerMut.Lock(pchannel)
|
||||
defer c.managerMut.Unlock(pchannel)
|
||||
var manager DispatcherManager
|
||||
manager, ok := c.managers[pchannel]
|
||||
manager, ok := c.managers.Get(pchannel)
|
||||
if !ok {
|
||||
manager = NewDispatcherManager(pchannel, c.role, c.nodeID, c.factory)
|
||||
c.managers[pchannel] = manager
|
||||
c.managers.Insert(pchannel, manager)
|
||||
go manager.Run()
|
||||
}
|
||||
// Check if the consumer number limit has been reached.
|
||||
if manager.Num() >= paramtable.Get().MQCfg.MaxDispatcherNumPerPchannel.GetAsInt() {
|
||||
return nil, ErrTooManyConsumers
|
||||
}
|
||||
// Begin to register
|
||||
ch, err := manager.Add(ctx, vchannel, pos, subPos)
|
||||
if err != nil {
|
||||
if manager.Num() == 0 {
|
||||
manager.Close()
|
||||
delete(c.managers, pchannel)
|
||||
c.managers.Remove(pchannel)
|
||||
}
|
||||
log.Error("register failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
log.Info("register done")
|
||||
log.Info("register done", zap.Duration("dur", time.Since(start)))
|
||||
return ch, nil
|
||||
}
|
||||
|
||||
func (c *client) Deregister(vchannel string) {
|
||||
pchannel := funcutil.ToPhysicalChannel(vchannel)
|
||||
c.managerMut.Lock()
|
||||
defer c.managerMut.Unlock()
|
||||
if manager, ok := c.managers[pchannel]; ok {
|
||||
start := time.Now()
|
||||
c.managerMut.Lock(pchannel)
|
||||
defer c.managerMut.Unlock(pchannel)
|
||||
if manager, ok := c.managers.Get(pchannel); ok {
|
||||
manager.Remove(vchannel)
|
||||
if manager.Num() == 0 {
|
||||
manager.Close()
|
||||
delete(c.managers, pchannel)
|
||||
c.managers.Remove(pchannel)
|
||||
}
|
||||
log.Info("deregister done", zap.String("role", c.role),
|
||||
zap.Int64("nodeID", c.nodeID), zap.String("vchannel", vchannel))
|
||||
log.Info("deregister done", zap.String("role", c.role), zap.Int64("nodeID", c.nodeID),
|
||||
zap.String("vchannel", vchannel), zap.Duration("dur", time.Since(start)))
|
||||
}
|
||||
}
|
||||
|
||||
func (c *client) Close() {
|
||||
log := log.With(zap.String("role", c.role),
|
||||
zap.Int64("nodeID", c.nodeID))
|
||||
c.managerMut.Lock()
|
||||
defer c.managerMut.Unlock()
|
||||
for pchannel, manager := range c.managers {
|
||||
|
||||
c.managers.Range(func(pchannel string, manager DispatcherManager) bool {
|
||||
c.managerMut.Lock(pchannel)
|
||||
defer c.managerMut.Unlock(pchannel)
|
||||
log.Info("close manager", zap.String("channel", pchannel))
|
||||
delete(c.managers, pchannel)
|
||||
c.managers.Remove(pchannel)
|
||||
manager.Close()
|
||||
}
|
||||
return true
|
||||
})
|
||||
log.Info("dispatcher client closed")
|
||||
}
|
||||
|
|
|
@ -79,8 +79,6 @@ func TestClient_Concurrency(t *testing.T) {
|
|||
expected := int(total - deregisterCount.Load())
|
||||
|
||||
c := client1.(*client)
|
||||
c.managerMut.Lock()
|
||||
n := len(c.managers)
|
||||
c.managerMut.Unlock()
|
||||
n := c.managers.Len()
|
||||
assert.Equal(t, expected, n)
|
||||
}
|
||||
|
|
|
@ -527,9 +527,12 @@ type MQConfig struct {
|
|||
IgnoreBadPosition ParamItem `refreshable:"true"`
|
||||
|
||||
// msgdispatcher
|
||||
MergeCheckInterval ParamItem `refreshable:"false"`
|
||||
TargetBufSize ParamItem `refreshable:"false"`
|
||||
MaxTolerantLag ParamItem `refreshable:"true"`
|
||||
MergeCheckInterval ParamItem `refreshable:"false"`
|
||||
TargetBufSize ParamItem `refreshable:"false"`
|
||||
MaxTolerantLag ParamItem `refreshable:"true"`
|
||||
MaxDispatcherNumPerPchannel ParamItem `refreshable:"true"`
|
||||
RetrySleep ParamItem `refreshable:"true"`
|
||||
RetryTimeout ParamItem `refreshable:"true"`
|
||||
}
|
||||
|
||||
// Init initializes the MQConfig object with a BaseTable.
|
||||
|
@ -553,6 +556,35 @@ Valid values: [default, pulsar, kafka, rocksmq, natsmq]`,
|
|||
}
|
||||
p.MaxTolerantLag.Init(base.mgr)
|
||||
|
||||
p.MaxDispatcherNumPerPchannel = ParamItem{
|
||||
Key: "mq.dispatcher.maxDispatcherNumPerPchannel",
|
||||
Version: "2.4.19",
|
||||
DefaultValue: "10",
|
||||
Doc: ` The maximum number of dispatchers per physical channel, primarily to limit
|
||||
the number of MQ consumers and prevent performance issues
|
||||
(e.g., during recovery when a large number of channels are watched).`,
|
||||
Export: true,
|
||||
}
|
||||
p.MaxDispatcherNumPerPchannel.Init(base.mgr)
|
||||
|
||||
p.RetrySleep = ParamItem{
|
||||
Key: "mq.dispatcher.retrySleep",
|
||||
Version: "2.4.19",
|
||||
DefaultValue: "5",
|
||||
Doc: `register retry sleep time in seconds`,
|
||||
Export: true,
|
||||
}
|
||||
p.RetrySleep.Init(base.mgr)
|
||||
|
||||
p.RetryTimeout = ParamItem{
|
||||
Key: "mq.dispatcher.retryTimeout",
|
||||
Version: "2.4.19",
|
||||
DefaultValue: "300",
|
||||
Doc: `register retry timeout in seconds`,
|
||||
Export: true,
|
||||
}
|
||||
p.RetryTimeout.Init(base.mgr)
|
||||
|
||||
p.TargetBufSize = ParamItem{
|
||||
Key: "mq.dispatcher.targetBufSize",
|
||||
Version: "2.4.4",
|
||||
|
|
|
@ -26,6 +26,17 @@ func TestServiceParam(t *testing.T) {
|
|||
var SParams ServiceParam
|
||||
bt := NewBaseTable(SkipRemote(true))
|
||||
SParams.init(bt)
|
||||
|
||||
t.Run("test MQConfig", func(t *testing.T) {
|
||||
Params := &SParams.MQCfg
|
||||
assert.Equal(t, 1*time.Second, Params.MergeCheckInterval.GetAsDuration(time.Second))
|
||||
assert.Equal(t, 16, Params.TargetBufSize.GetAsInt())
|
||||
assert.Equal(t, 3*time.Second, Params.MaxTolerantLag.GetAsDuration(time.Second))
|
||||
assert.Equal(t, 10, Params.MaxDispatcherNumPerPchannel.GetAsInt())
|
||||
assert.Equal(t, 5*time.Second, Params.RetrySleep.GetAsDuration(time.Second))
|
||||
assert.Equal(t, 300*time.Second, Params.RetryTimeout.GetAsDuration(time.Second))
|
||||
})
|
||||
|
||||
t.Run("test etcdConfig", func(t *testing.T) {
|
||||
Params := &SParams.EtcdCfg
|
||||
|
||||
|
|
Loading…
Reference in New Issue