From a5a83a090416c6be218309441aa9aac31d316298 Mon Sep 17 00:00:00 2001 From: "yihao.dai" Date: Wed, 15 Jan 2025 21:37:01 +0800 Subject: [PATCH] fix: Fix consume blocked due to too many consumers (#38455) 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 --------- Signed-off-by: bigsheeper --- configs/milvus.yaml | 4 +- internal/datacoord/channel_manager.go | 14 +---- .../flow_graph_dmstream_input_node.go | 58 ++++++++++++++----- internal/util/pipeline/stream_pipeline.go | 27 ++++++--- .../util/pipeline/stream_pipeline_test.go | 2 + pkg/mq/msgdispatcher/client.go | 18 +++++- pkg/util/merr/errors.go | 9 +-- pkg/util/merr/errors_test.go | 1 + pkg/util/merr/utils.go | 8 +++ pkg/util/paramtable/component_param.go | 22 ++----- pkg/util/paramtable/component_param_test.go | 1 - pkg/util/paramtable/service_param.go | 36 +++++++++++- pkg/util/paramtable/service_param_test.go | 11 ++++ 13 files changed, 150 insertions(+), 61 deletions(-) diff --git a/configs/milvus.yaml b/configs/milvus.yaml index 57563e32e0..7023f7f04f 100644 --- a/configs/milvus.yaml +++ b/configs/milvus.yaml @@ -176,6 +176,9 @@ 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 + maxDispatcherNumPerPchannel: 5 # The maximum number of dispatchers per physical channel, primarily to limit the number of consumers and prevent performance issues(e.g., during recovery when a large number of channels are watched). + retrySleep: 3 # register retry sleep time in seconds + retryTimeout: 60 # 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: @@ -520,7 +523,6 @@ dataCoord: balanceInterval: 360 # The interval with which the channel manager check dml channel balance status checkInterval: 1 # The interval in seconds with which the channel manager advances channel states notifyChannelOperationTimeout: 5 # Timeout notifing channel operations (in seconds). - maxConcurrentChannelTaskNumPerDN: 32 # The maximum concurrency for each DataNode executing channel tasks (watch, release). segment: maxSize: 1024 # The maximum size of a segment, unit: MB. datacoord.segment.maxSize and datacoord.segment.sealProportion together determine if a segment can be sealed. diskSegmentMaxSize: 2048 # Maximun size of a segment in MB for collection which has Disk index diff --git a/internal/datacoord/channel_manager.go b/internal/datacoord/channel_manager.go index b75134489d..a5d9c016be 100644 --- a/internal/datacoord/channel_manager.go +++ b/internal/datacoord/channel_manager.go @@ -35,7 +35,6 @@ import ( "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/lock" "github.com/milvus-io/milvus/pkg/util/merr" - "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -451,24 +450,13 @@ func (m *ChannelManagerImpl) AdvanceChannelState(ctx context.Context) { standbys := m.store.GetNodeChannelsBy(WithAllNodes(), WithChannelStates(Standby)) toNotifies := m.store.GetNodeChannelsBy(WithoutBufferNode(), WithChannelStates(ToWatch, ToRelease)) toChecks := m.store.GetNodeChannelsBy(WithoutBufferNode(), WithChannelStates(Watching, Releasing)) - maxNum := len(m.store.GetNodes()) * paramtable.Get().DataCoordCfg.MaxConcurrentChannelTaskNumPerDN.GetAsInt() m.mu.RUnlock() // Processing standby channels updatedStandbys := false updatedStandbys = m.advanceStandbys(ctx, standbys) updatedToCheckes := m.advanceToChecks(ctx, toChecks) - - var ( - updatedToNotifies bool - executingNum = len(toChecks) - toNotifyNum = maxNum - executingNum - ) - - if toNotifyNum > 0 { - toNotifies = lo.Slice(toNotifies, 0, toNotifyNum) - updatedToNotifies = m.advanceToNotifies(ctx, toNotifies) - } + updatedToNotifies := m.advanceToNotifies(ctx, toNotifies) if updatedStandbys || updatedToCheckes || updatedToNotifies { m.lastActiveTimestamp = time.Now() diff --git a/internal/flushcommon/pipeline/flow_graph_dmstream_input_node.go b/internal/flushcommon/pipeline/flow_graph_dmstream_input_node.go index 2aa8e3927b..8ed0876288 100644 --- a/internal/flushcommon/pipeline/flow_graph_dmstream_input_node.go +++ b/internal/flushcommon/pipeline/flow_graph_dmstream_input_node.go @@ -21,6 +21,7 @@ import ( "fmt" "time" + "github.com/cockroachdb/errors" "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" @@ -33,7 +34,9 @@ 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/merr" "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" ) @@ -69,6 +72,13 @@ func createNewInputFromDispatcher(initCtx context.Context, ) (<-chan *msgstream.MsgPack, error) { log := log.With(zap.Int64("nodeID", paramtable.GetNodeID()), zap.String("vchannel", vchannel)) + + var ( + input <-chan *msgstream.MsgPack + err error + start = time.Now() + ) + replicateID, _ := pkgcommon.GetReplicateID(schema.GetProperties()) if replicateID == "" { log.Info("datanode consume without replicateID, try to get replicateID from dbProperties", zap.Any("dbProperties", dbProperties)) @@ -77,30 +87,52 @@ func createNewInputFromDispatcher(initCtx context.Context, replicateConfig := msgstream.GetReplicateConfig(replicateID, schema.GetDbName(), schema.GetName()) if seekPos != nil && len(seekPos.MsgID) != 0 { - input, err := dispatcherClient.Register(initCtx, &msgdispatcher.StreamConfig{ - VChannel: vchannel, - Pos: seekPos, - SubPos: common.SubscriptionPositionUnknown, - ReplicateConfig: replicateConfig, - }) + err := retry.Handle(initCtx, func() (bool, error) { + input, err = dispatcherClient.Register(initCtx, &msgdispatcher.StreamConfig{ + VChannel: vchannel, + Pos: seekPos, + SubPos: common.SubscriptionPositionUnknown, + ReplicateConfig: replicateConfig, + }) + if err != nil { + log.Warn("datanode consume failed", zap.Error(err)) + return errors.Is(err, merr.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))) return input, err } - input, err := dispatcherClient.Register(initCtx, &msgdispatcher.StreamConfig{ - VChannel: vchannel, - Pos: nil, - SubPos: common.SubscriptionPositionEarliest, - ReplicateConfig: replicateConfig, - }) + + err = retry.Handle(initCtx, func() (bool, error) { + input, err = dispatcherClient.Register(initCtx, &msgdispatcher.StreamConfig{ + VChannel: vchannel, + Pos: nil, + SubPos: common.SubscriptionPositionEarliest, + ReplicateConfig: replicateConfig, + }) + if err != nil { + log.Warn("datanode consume failed", zap.Error(err)) + return errors.Is(err, merr.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") return input, err } diff --git a/internal/util/pipeline/stream_pipeline.go b/internal/util/pipeline/stream_pipeline.go index 888f1a9da0..ce60350b91 100644 --- a/internal/util/pipeline/stream_pipeline.go +++ b/internal/util/pipeline/stream_pipeline.go @@ -22,6 +22,7 @@ import ( "sync" "time" + "github.com/cockroachdb/errors" "go.uber.org/atomic" "go.uber.org/zap" @@ -35,6 +36,9 @@ import ( "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/message/adaptor" "github.com/milvus-io/milvus/pkg/streaming/util/options" + "github.com/milvus-io/milvus/pkg/util/merr" + "github.com/milvus-io/milvus/pkg/util/paramtable" + "github.com/milvus-io/milvus/pkg/util/retry" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) @@ -119,16 +123,25 @@ func (p *streamPipeline) ConsumeMsgStream(ctx context.Context, position *msgpb.M } start := time.Now() - p.input, err = p.dispatcher.Register(ctx, &msgdispatcher.StreamConfig{ - VChannel: p.vChannel, - Pos: position, - SubPos: common.SubscriptionPositionUnknown, - ReplicateConfig: p.replicateConfig, - }) + err = retry.Handle(ctx, func() (bool, error) { + p.input, err = p.dispatcher.Register(ctx, &msgdispatcher.StreamConfig{ + VChannel: p.vChannel, + Pos: position, + SubPos: common.SubscriptionPositionUnknown, + ReplicateConfig: p.replicateConfig, + }) + if err != nil { + log.Warn("dispatcher register failed", zap.String("channel", position.ChannelName), zap.Error(err)) + return errors.Is(err, merr.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), diff --git a/internal/util/pipeline/stream_pipeline_test.go b/internal/util/pipeline/stream_pipeline_test.go index 1b28b558e8..41154f3728 100644 --- a/internal/util/pipeline/stream_pipeline_test.go +++ b/internal/util/pipeline/stream_pipeline_test.go @@ -27,6 +27,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/util/paramtable" ) type StreamPipelineSuite struct { @@ -42,6 +43,7 @@ type StreamPipelineSuite struct { } func (suite *StreamPipelineSuite) SetupTest() { + paramtable.Init() suite.channel = "test-channel" suite.inChannel = make(chan *msgstream.MsgPack, 1) suite.outChannel = make(chan msgstream.Timestamp) diff --git a/pkg/mq/msgdispatcher/client.go b/pkg/mq/msgdispatcher/client.go index 7875c67dab..79dcef0732 100644 --- a/pkg/mq/msgdispatcher/client.go +++ b/pkg/mq/msgdispatcher/client.go @@ -18,6 +18,8 @@ package msgdispatcher import ( "context" + "fmt" + "time" "go.uber.org/zap" @@ -27,6 +29,8 @@ import ( "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/merr" + "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -82,6 +86,7 @@ func (c *client) Register(ctx context.Context, streamConfig *StreamConfig) (<-ch log := log.With(zap.String("role", c.role), zap.Int64("nodeID", c.nodeID), zap.String("vchannel", vchannel)) pchannel := funcutil.ToPhysicalChannel(vchannel) + start := time.Now() c.managerMut.Lock(pchannel) defer c.managerMut.Unlock(pchannel) var manager DispatcherManager @@ -91,6 +96,12 @@ func (c *client) Register(ctx context.Context, streamConfig *StreamConfig) (<-ch c.managers.Insert(pchannel, manager) go manager.Run() } + // Check if the consumer number limit has been reached. + limit := paramtable.Get().MQCfg.MaxDispatcherNumPerPchannel.GetAsInt() + if manager.Num() >= limit { + return nil, merr.WrapErrTooManyConsumers(vchannel, fmt.Sprintf("limit=%d", limit)) + } + // Begin to register ch, err := manager.Add(ctx, streamConfig) if err != nil { if manager.Num() == 0 { @@ -100,12 +111,13 @@ func (c *client) Register(ctx context.Context, streamConfig *StreamConfig) (<-ch 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) + start := time.Now() c.managerMut.Lock(pchannel) defer c.managerMut.Unlock(pchannel) if manager, ok := c.managers.Get(pchannel); ok { @@ -114,8 +126,8 @@ func (c *client) Deregister(vchannel string) { manager.Close() 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))) } } diff --git a/pkg/util/merr/errors.go b/pkg/util/merr/errors.go index b36d7230b9..505ba7d354 100644 --- a/pkg/util/merr/errors.go +++ b/pkg/util/merr/errors.go @@ -140,10 +140,11 @@ var ( ErrMetricNotFound = newMilvusError("metric not found", 1200, false) // Message queue related - ErrMqTopicNotFound = newMilvusError("topic not found", 1300, false) - ErrMqTopicNotEmpty = newMilvusError("topic not empty", 1301, false) - ErrMqInternal = newMilvusError("message queue internal error", 1302, false) - ErrDenyProduceMsg = newMilvusError("deny to write the message to mq", 1303, false) + ErrMqTopicNotFound = newMilvusError("topic not found", 1300, false) + ErrMqTopicNotEmpty = newMilvusError("topic not empty", 1301, false) + ErrMqInternal = newMilvusError("message queue internal error", 1302, false) + ErrDenyProduceMsg = newMilvusError("deny to write the message to mq", 1303, false) + ErrTooManyConsumers = newMilvusError("consumer number limit exceeded", 1304, false) // Privilege related // this operation is denied because the user not authorized, user need to login in first diff --git a/pkg/util/merr/errors_test.go b/pkg/util/merr/errors_test.go index 4febe3e64c..d31dd11d68 100644 --- a/pkg/util/merr/errors_test.go +++ b/pkg/util/merr/errors_test.go @@ -147,6 +147,7 @@ func (s *ErrSuite) TestWrap() { s.ErrorIs(WrapErrMqTopicNotFound("unknown", "failed to get topic"), ErrMqTopicNotFound) s.ErrorIs(WrapErrMqTopicNotEmpty("unknown", "topic is not empty"), ErrMqTopicNotEmpty) s.ErrorIs(WrapErrMqInternal(errors.New("unknown"), "failed to consume"), ErrMqInternal) + s.ErrorIs(WrapErrTooManyConsumers("unknown", "too many consumers"), ErrTooManyConsumers) // field related s.ErrorIs(WrapErrFieldNotFound("meta", "failed to get field"), ErrFieldNotFound) diff --git a/pkg/util/merr/utils.go b/pkg/util/merr/utils.go index f86711e729..f40681486f 100644 --- a/pkg/util/merr/utils.go +++ b/pkg/util/merr/utils.go @@ -989,6 +989,14 @@ func WrapErrMqInternal(err error, msg ...string) error { return err } +func WrapErrTooManyConsumers(vchannel string, msg ...string) error { + err := wrapFields(ErrTooManyConsumers, value("vchannel", vchannel)) + if len(msg) > 0 { + err = errors.Wrap(err, strings.Join(msg, "->")) + } + return err +} + func WrapErrPrivilegeNotAuthenticated(fmt string, args ...any) error { err := errors.Wrapf(ErrPrivilegeNotAuthenticated, fmt, args...) return err diff --git a/pkg/util/paramtable/component_param.go b/pkg/util/paramtable/component_param.go index 5bc153b84f..712668a9ca 100644 --- a/pkg/util/paramtable/component_param.go +++ b/pkg/util/paramtable/component_param.go @@ -3321,13 +3321,12 @@ user-task-polling: // --- datacoord --- type dataCoordConfig struct { // --- CHANNEL --- - WatchTimeoutInterval ParamItem `refreshable:"false"` - LegacyVersionWithoutRPCWatch ParamItem `refreshable:"false"` - ChannelBalanceSilentDuration ParamItem `refreshable:"true"` - ChannelBalanceInterval ParamItem `refreshable:"true"` - ChannelCheckInterval ParamItem `refreshable:"true"` - ChannelOperationRPCTimeout ParamItem `refreshable:"true"` - MaxConcurrentChannelTaskNumPerDN ParamItem `refreshable:"true"` + WatchTimeoutInterval ParamItem `refreshable:"false"` + LegacyVersionWithoutRPCWatch ParamItem `refreshable:"false"` + ChannelBalanceSilentDuration ParamItem `refreshable:"true"` + ChannelBalanceInterval ParamItem `refreshable:"true"` + ChannelCheckInterval ParamItem `refreshable:"true"` + ChannelOperationRPCTimeout ParamItem `refreshable:"true"` // --- SEGMENTS --- SegmentMaxSize ParamItem `refreshable:"false"` @@ -3499,15 +3498,6 @@ func (p *dataCoordConfig) init(base *BaseTable) { } p.ChannelOperationRPCTimeout.Init(base.mgr) - p.MaxConcurrentChannelTaskNumPerDN = ParamItem{ - Key: "dataCoord.channel.maxConcurrentChannelTaskNumPerDN", - Version: "2.5", - DefaultValue: "32", - Doc: "The maximum concurrency for each DataNode executing channel tasks (watch, release).", - Export: true, - } - p.MaxConcurrentChannelTaskNumPerDN.Init(base.mgr) - p.SegmentMaxSize = ParamItem{ Key: "dataCoord.segment.maxSize", Version: "2.0.0", diff --git a/pkg/util/paramtable/component_param_test.go b/pkg/util/paramtable/component_param_test.go index 7f1b5c0e25..4bc48f933f 100644 --- a/pkg/util/paramtable/component_param_test.go +++ b/pkg/util/paramtable/component_param_test.go @@ -539,7 +539,6 @@ func TestComponentParam(t *testing.T) { assert.Equal(t, 4, Params.L0DeleteCompactionSlotUsage.GetAsInt()) params.Save("datacoord.scheduler.taskSlowThreshold", "1000") assert.Equal(t, 1000*time.Second, Params.TaskSlowThreshold.GetAsDuration(time.Second)) - assert.Equal(t, 32, Params.MaxConcurrentChannelTaskNumPerDN.GetAsInt()) }) t.Run("test dataNodeConfig", func(t *testing.T) { diff --git a/pkg/util/paramtable/service_param.go b/pkg/util/paramtable/service_param.go index 42bff06595..d648611bb0 100644 --- a/pkg/util/paramtable/service_param.go +++ b/pkg/util/paramtable/service_param.go @@ -518,9 +518,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. @@ -544,6 +547,33 @@ Valid values: [default, pulsar, kafka, rocksmq, natsmq]`, } p.MaxTolerantLag.Init(base.mgr) + p.MaxDispatcherNumPerPchannel = ParamItem{ + Key: "mq.dispatcher.maxDispatcherNumPerPchannel", + Version: "2.4.19", + DefaultValue: "5", + Doc: `The maximum number of dispatchers per physical channel, primarily to limit the number of 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: "3", + 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: "60", + Doc: `register retry timeout in seconds`, + Export: true, + } + p.RetryTimeout.Init(base.mgr) + p.TargetBufSize = ParamItem{ Key: "mq.dispatcher.targetBufSize", Version: "2.4.4", diff --git a/pkg/util/paramtable/service_param_test.go b/pkg/util/paramtable/service_param_test.go index e8f8584e48..863b36a89f 100644 --- a/pkg/util/paramtable/service_param_test.go +++ b/pkg/util/paramtable/service_param_test.go @@ -32,6 +32,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, 5, Params.MaxDispatcherNumPerPchannel.GetAsInt()) + assert.Equal(t, 3*time.Second, Params.RetrySleep.GetAsDuration(time.Second)) + assert.Equal(t, 60*time.Second, Params.RetryTimeout.GetAsDuration(time.Second)) + }) + t.Run("test etcdConfig", func(t *testing.T) { Params := &SParams.EtcdCfg