fix: Balance segment/channel won't be trigger on multi replicas (#31107)

issue: #30983 #30982

cause balancer call wrong interface to get segment/channel list in
replica, then got a wrong average segment/channel number, which make
each node have less segment/channel than average, and the balance won't
be trigger in multi replica case.

This PR fix that balance segment/channel won't be trigger on multi
replicas

Signed-off-by: Wei Liu <wei.liu@zilliz.com>
pull/31182/head^2
wei liu 2024-03-11 20:35:04 +08:00 committed by GitHub
parent 6a83f16871
commit 06df9b8462
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
6 changed files with 500 additions and 3 deletions

View File

@ -271,7 +271,7 @@ func (b *RowCountBasedBalancer) genSegmentPlan(replica *meta.Replica, onlineNode
segmentsToMove = lo.Filter(segmentsToMove, func(s *meta.Segment, _ int) bool {
// if the segment are redundant, skip it's balance for now
return len(b.dist.SegmentDistManager.GetByFilter(meta.WithSegmentID(s.GetID()))) == 1
return len(b.dist.GetByFilter(meta.WithReplica(replica), meta.WithSegmentID(s.GetID()))) == 1
})
if len(nodesWithLessRow) == 0 || len(segmentsToMove) == 0 {
@ -305,7 +305,7 @@ func (b *RowCountBasedBalancer) genChannelPlan(replica *meta.Replica, onlineNode
channelPlans := make([]ChannelAssignPlan, 0)
if len(onlineNodes) > 1 {
// start to balance channels on all available nodes
channelDist := b.dist.ChannelDistManager.GetByCollection(replica.CollectionID)
channelDist := b.dist.ChannelDistManager.GetChannelDistByReplica(replica)
if len(channelDist) == 0 {
return nil
}

View File

@ -986,6 +986,7 @@ func (suite *RowCountBasedBalancerTestSuite) TestDisableBalanceChannel() {
}
Params.Save(Params.QueryCoordCfg.AutoBalanceChannel.Key, fmt.Sprint(c.enableBalanceChannel))
defer Params.Reset(Params.QueryCoordCfg.AutoBalanceChannel.Key)
segmentPlans, channelPlans := suite.getCollectionBalancePlans(balancer, 1)
if !c.multiple {
suite.ElementsMatch(c.expectChannelPlans, channelPlans)
@ -1006,6 +1007,130 @@ func (suite *RowCountBasedBalancerTestSuite) TestDisableBalanceChannel() {
}
}
func (suite *RowCountBasedBalancerTestSuite) TestMultiReplicaBalance() {
cases := []struct {
name string
collectionID int64
replicaWithNodes map[int64][]int64
segments []*datapb.SegmentInfo
channels []*datapb.VchannelInfo
states []session.State
shouldMock bool
segmentDist map[int64][]*meta.Segment
channelDist map[int64][]*meta.DmChannel
expectPlans []SegmentAssignPlan
expectChannelPlans []ChannelAssignPlan
}{
{
name: "balance on multi replica",
collectionID: 1,
replicaWithNodes: map[int64][]int64{1: {1, 2}, 2: {3, 4}},
segments: []*datapb.SegmentInfo{
{ID: 1, CollectionID: 1, PartitionID: 1},
{ID: 2, CollectionID: 1, PartitionID: 1},
{ID: 3, CollectionID: 1, PartitionID: 1},
{ID: 4, CollectionID: 1, PartitionID: 1},
},
channels: []*datapb.VchannelInfo{
{
CollectionID: 1, ChannelName: "channel1", FlushedSegmentIds: []int64{1},
},
{
CollectionID: 1, ChannelName: "channel2", FlushedSegmentIds: []int64{2},
},
{
CollectionID: 1, ChannelName: "channel3", FlushedSegmentIds: []int64{3},
},
{
CollectionID: 1, ChannelName: "channel4", FlushedSegmentIds: []int64{4},
},
},
states: []session.State{session.NodeStateNormal, session.NodeStateNormal},
segmentDist: map[int64][]*meta.Segment{
1: {
{SegmentInfo: &datapb.SegmentInfo{ID: 1, CollectionID: 1, NumOfRows: 30}, Node: 1},
{SegmentInfo: &datapb.SegmentInfo{ID: 2, CollectionID: 1, NumOfRows: 30}, Node: 1},
},
3: {
{SegmentInfo: &datapb.SegmentInfo{ID: 3, CollectionID: 1, NumOfRows: 30}, Node: 3},
{SegmentInfo: &datapb.SegmentInfo{ID: 4, CollectionID: 1, NumOfRows: 30}, Node: 3},
},
},
channelDist: map[int64][]*meta.DmChannel{
1: {
{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1},
{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 1},
},
3: {
{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3},
{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 3},
},
},
expectPlans: []SegmentAssignPlan{},
expectChannelPlans: []ChannelAssignPlan{},
},
}
for _, c := range cases {
suite.Run(c.name, func() {
suite.SetupSuite()
defer suite.TearDownTest()
balancer := suite.balancer
// 1. set up target for multi collections
collection := utils.CreateTestCollection(c.collectionID, int32(len(c.replicaWithNodes)))
suite.broker.EXPECT().GetRecoveryInfoV2(mock.Anything, c.collectionID).Return(
c.channels, c.segments, nil)
suite.broker.EXPECT().GetPartitions(mock.Anything, c.collectionID).Return([]int64{c.collectionID}, nil).Maybe()
collection.LoadPercentage = 100
collection.Status = querypb.LoadStatus_Loaded
balancer.meta.CollectionManager.PutCollection(collection)
balancer.meta.CollectionManager.PutPartition(utils.CreateTestPartition(c.collectionID, c.collectionID))
for replicaID, nodes := range c.replicaWithNodes {
balancer.meta.ReplicaManager.Put(utils.CreateTestReplica(replicaID, c.collectionID, nodes))
}
balancer.targetMgr.UpdateCollectionNextTarget(c.collectionID)
balancer.targetMgr.UpdateCollectionCurrentTarget(c.collectionID)
balancer.targetMgr.UpdateCollectionNextTarget(c.collectionID)
// 2. set up target for distribution for multi collections
for node, s := range c.segmentDist {
balancer.dist.SegmentDistManager.Update(node, s...)
}
for node, v := range c.channelDist {
balancer.dist.ChannelDistManager.Update(node, v...)
}
// 3. set up nodes info and resourceManager for balancer
for _, nodes := range c.replicaWithNodes {
for i := range nodes {
nodeInfo := session.NewNodeInfo(nodes[i], "127.0.0.1:0")
nodeInfo.UpdateStats(session.WithChannelCnt(len(c.channelDist[nodes[i]])))
nodeInfo.SetState(c.states[i])
suite.balancer.nodeManager.Add(nodeInfo)
suite.balancer.meta.ResourceManager.AssignNode(meta.DefaultResourceGroupName, nodes[i])
}
}
// expected to balance channel first
segmentPlans, channelPlans := suite.getCollectionBalancePlans(balancer, c.collectionID)
suite.Len(segmentPlans, 0)
suite.Len(channelPlans, 2)
// mock new distribution after channel balance
balancer.dist.ChannelDistManager.Update(1, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1})
balancer.dist.ChannelDistManager.Update(2, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 2})
balancer.dist.ChannelDistManager.Update(3, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3})
balancer.dist.ChannelDistManager.Update(4, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 4})
// expected to balance segment
segmentPlans, channelPlans = suite.getCollectionBalancePlans(balancer, c.collectionID)
suite.Len(segmentPlans, 2)
suite.Len(channelPlans, 0)
})
}
}
func TestRowCountBasedBalancerSuite(t *testing.T) {
suite.Run(t, new(RowCountBasedBalancerTestSuite))
}

View File

@ -99,6 +99,9 @@ func (b *ScoreBasedBalancer) AssignSegment(collectionID int64, segments []*meta.
plans = append(plans, plan)
// update the targetNode's score
if sourceNode != nil {
sourceNode.setPriority(sourceNode.getPriority() - priorityChange)
}
targetNode.setPriority(targetNode.getPriority() + priorityChange)
}(s)
}
@ -298,7 +301,7 @@ func (b *ScoreBasedBalancer) genSegmentPlan(replica *meta.Replica, onlineNodes [
// if the segment are redundant, skip it's balance for now
segmentsToMove = lo.Filter(segmentsToMove, func(s *meta.Segment, _ int) bool {
return len(b.dist.SegmentDistManager.GetByFilter(meta.WithSegmentID(s.GetID()))) == 1
return len(b.dist.GetByFilter(meta.WithReplica(replica), meta.WithSegmentID(s.GetID()))) == 1
})
if len(segmentsToMove) == 0 {

View File

@ -637,6 +637,130 @@ func (suite *ScoreBasedBalancerTestSuite) TestStoppedBalance() {
}
}
func (suite *ScoreBasedBalancerTestSuite) TestMultiReplicaBalance() {
cases := []struct {
name string
collectionID int64
replicaWithNodes map[int64][]int64
segments []*datapb.SegmentInfo
channels []*datapb.VchannelInfo
states []session.State
shouldMock bool
segmentDist map[int64][]*meta.Segment
channelDist map[int64][]*meta.DmChannel
expectPlans []SegmentAssignPlan
expectChannelPlans []ChannelAssignPlan
}{
{
name: "normal balance for one collection only",
collectionID: 1,
replicaWithNodes: map[int64][]int64{1: {1, 2}, 2: {3, 4}},
segments: []*datapb.SegmentInfo{
{ID: 1, CollectionID: 1, PartitionID: 1},
{ID: 2, CollectionID: 1, PartitionID: 1},
{ID: 3, CollectionID: 1, PartitionID: 1},
{ID: 4, CollectionID: 1, PartitionID: 1},
},
channels: []*datapb.VchannelInfo{
{
CollectionID: 1, ChannelName: "channel1", FlushedSegmentIds: []int64{1},
},
{
CollectionID: 1, ChannelName: "channel2", FlushedSegmentIds: []int64{2},
},
{
CollectionID: 1, ChannelName: "channel3", FlushedSegmentIds: []int64{3},
},
{
CollectionID: 1, ChannelName: "channel4", FlushedSegmentIds: []int64{4},
},
},
states: []session.State{session.NodeStateNormal, session.NodeStateNormal},
segmentDist: map[int64][]*meta.Segment{
1: {
{SegmentInfo: &datapb.SegmentInfo{ID: 1, CollectionID: 1, NumOfRows: 30}, Node: 1},
{SegmentInfo: &datapb.SegmentInfo{ID: 2, CollectionID: 1, NumOfRows: 30}, Node: 1},
},
3: {
{SegmentInfo: &datapb.SegmentInfo{ID: 3, CollectionID: 1, NumOfRows: 30}, Node: 3},
{SegmentInfo: &datapb.SegmentInfo{ID: 4, CollectionID: 1, NumOfRows: 30}, Node: 3},
},
},
channelDist: map[int64][]*meta.DmChannel{
1: {
{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1},
{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 1},
},
3: {
{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3},
{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 3},
},
},
expectPlans: []SegmentAssignPlan{},
expectChannelPlans: []ChannelAssignPlan{},
},
}
for _, c := range cases {
suite.Run(c.name, func() {
suite.SetupSuite()
defer suite.TearDownTest()
balancer := suite.balancer
// 1. set up target for multi collections
collection := utils.CreateTestCollection(c.collectionID, int32(len(c.replicaWithNodes)))
suite.broker.EXPECT().GetRecoveryInfoV2(mock.Anything, c.collectionID).Return(
c.channels, c.segments, nil)
suite.broker.EXPECT().GetPartitions(mock.Anything, c.collectionID).Return([]int64{c.collectionID}, nil).Maybe()
collection.LoadPercentage = 100
collection.Status = querypb.LoadStatus_Loaded
balancer.meta.CollectionManager.PutCollection(collection)
balancer.meta.CollectionManager.PutPartition(utils.CreateTestPartition(c.collectionID, c.collectionID))
for replicaID, nodes := range c.replicaWithNodes {
balancer.meta.ReplicaManager.Put(utils.CreateTestReplica(replicaID, c.collectionID, nodes))
}
balancer.targetMgr.UpdateCollectionNextTarget(c.collectionID)
balancer.targetMgr.UpdateCollectionCurrentTarget(c.collectionID)
balancer.targetMgr.UpdateCollectionNextTarget(c.collectionID)
// 2. set up target for distribution for multi collections
for node, s := range c.segmentDist {
balancer.dist.SegmentDistManager.Update(node, s...)
}
for node, v := range c.channelDist {
balancer.dist.ChannelDistManager.Update(node, v...)
}
// 3. set up nodes info and resourceManager for balancer
for _, nodes := range c.replicaWithNodes {
for i := range nodes {
nodeInfo := session.NewNodeInfo(nodes[i], "127.0.0.1:0")
nodeInfo.UpdateStats(session.WithChannelCnt(len(c.channelDist[nodes[i]])))
nodeInfo.SetState(c.states[i])
suite.balancer.nodeManager.Add(nodeInfo)
suite.balancer.meta.ResourceManager.AssignNode(meta.DefaultResourceGroupName, nodes[i])
}
}
// expected to balance channel first
segmentPlans, channelPlans := suite.getCollectionBalancePlans(balancer, c.collectionID)
suite.Len(segmentPlans, 0)
suite.Len(channelPlans, 2)
// mock new distribution after channel balance
balancer.dist.ChannelDistManager.Update(1, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1})
balancer.dist.ChannelDistManager.Update(2, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 2})
balancer.dist.ChannelDistManager.Update(3, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3})
balancer.dist.ChannelDistManager.Update(4, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 4})
// expected to balance segment
segmentPlans, channelPlans = suite.getCollectionBalancePlans(balancer, c.collectionID)
suite.Len(segmentPlans, 2)
suite.Len(channelPlans, 0)
})
}
}
func TestScoreBasedBalancerSuite(t *testing.T) {
suite.Run(t, new(ScoreBasedBalancerTestSuite))
}

View File

@ -0,0 +1,238 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package balance
import (
"context"
"testing"
"time"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/suite"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metric"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/tests/integration"
)
type BalanceTestSuit struct {
integration.MiniClusterSuite
}
func (s *BalanceTestSuit) SetupSuite() {
paramtable.Init()
paramtable.Get().Save(paramtable.Get().QueryCoordCfg.BalanceCheckInterval.Key, "1000")
paramtable.Get().Save(paramtable.Get().QueryNodeCfg.GracefulStopTimeout.Key, "1")
s.Require().NoError(s.SetupEmbedEtcd())
}
func (s *BalanceTestSuit) initCollection(collectionName string, replica int, channelNum int, segmentNum int, segmentRowNum int) {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
const (
dim = 128
dbName = ""
)
schema := integration.ConstructSchema(collectionName, dim, true)
marshaledSchema, err := proto.Marshal(schema)
s.NoError(err)
createCollectionStatus, err := s.Cluster.Proxy.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{
DbName: dbName,
CollectionName: collectionName,
Schema: marshaledSchema,
ShardsNum: int32(channelNum),
})
s.NoError(err)
s.True(merr.Ok(createCollectionStatus))
log.Info("CreateCollection result", zap.Any("createCollectionStatus", createCollectionStatus))
showCollectionsResp, err := s.Cluster.Proxy.ShowCollections(ctx, &milvuspb.ShowCollectionsRequest{})
s.NoError(err)
s.True(merr.Ok(showCollectionsResp.Status))
log.Info("ShowCollections result", zap.Any("showCollectionsResp", showCollectionsResp))
for i := 0; i < segmentNum; i++ {
fVecColumn := integration.NewFloatVectorFieldData(integration.FloatVecField, segmentRowNum, dim)
hashKeys := integration.GenerateHashKeys(segmentRowNum)
insertResult, err := s.Cluster.Proxy.Insert(ctx, &milvuspb.InsertRequest{
DbName: dbName,
CollectionName: collectionName,
FieldsData: []*schemapb.FieldData{fVecColumn},
HashKeys: hashKeys,
NumRows: uint32(segmentRowNum),
})
s.NoError(err)
s.True(merr.Ok(insertResult.Status))
// flush
flushResp, err := s.Cluster.Proxy.Flush(ctx, &milvuspb.FlushRequest{
DbName: dbName,
CollectionNames: []string{collectionName},
})
s.NoError(err)
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
ids := segmentIDs.GetData()
s.Require().NotEmpty(segmentIDs)
s.Require().True(has)
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
s.True(has)
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
}
// create index
createIndexStatus, err := s.Cluster.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{
CollectionName: collectionName,
FieldName: integration.FloatVecField,
IndexName: "_default",
ExtraParams: integration.ConstructIndexParam(dim, integration.IndexFaissIvfFlat, metric.L2),
})
s.NoError(err)
s.True(merr.Ok(createIndexStatus))
s.WaitForIndexBuilt(ctx, collectionName, integration.FloatVecField)
for i := 1; i < replica; i++ {
s.Cluster.AddQueryNode()
}
// load
loadStatus, err := s.Cluster.Proxy.LoadCollection(ctx, &milvuspb.LoadCollectionRequest{
DbName: dbName,
CollectionName: collectionName,
ReplicaNumber: int32(replica),
})
s.NoError(err)
s.Equal(commonpb.ErrorCode_Success, loadStatus.GetErrorCode())
s.True(merr.Ok(loadStatus))
s.WaitForLoad(ctx, collectionName)
log.Info("initCollection Done")
}
func (s *BalanceTestSuit) TestBalanceOnSingleReplica() {
name := "test_balance_" + funcutil.GenRandomStr()
s.initCollection(name, 1, 2, 2, 2000)
ctx := context.Background()
// disable compact
s.Cluster.DataCoord.GcControl(ctx, &datapb.GcControlRequest{
Base: commonpbutil.NewMsgBase(),
Command: datapb.GcCommand_Pause,
Params: []*commonpb.KeyValuePair{
{Key: "duration", Value: "3600"},
},
})
defer s.Cluster.DataCoord.GcControl(ctx, &datapb.GcControlRequest{
Base: commonpbutil.NewMsgBase(),
Command: datapb.GcCommand_Resume,
})
// add a querynode, expected balance happens
qn := s.Cluster.AddQueryNode()
// check segment number on new querynode
s.Eventually(func() bool {
resp, err := qn.GetDataDistribution(ctx, &querypb.GetDataDistributionRequest{})
s.NoError(err)
s.True(merr.Ok(resp.GetStatus()))
return len(resp.Channels) == 1 && len(resp.Segments) == 2
}, 30*time.Second, 1*time.Second)
// check total segment number
s.Eventually(func() bool {
count := 0
for _, node := range s.Cluster.GetAllQueryNodes() {
resp1, err := node.GetDataDistribution(ctx, &querypb.GetDataDistributionRequest{})
s.NoError(err)
s.True(merr.Ok(resp1.GetStatus()))
count += len(resp1.Segments)
}
return count == 4
}, 10*time.Second, 1*time.Second)
}
func (s *BalanceTestSuit) TestBalanceOnMultiReplica() {
ctx := context.Background()
// disable compact
s.Cluster.DataCoord.GcControl(ctx, &datapb.GcControlRequest{
Base: commonpbutil.NewMsgBase(),
Command: datapb.GcCommand_Pause,
Params: []*commonpb.KeyValuePair{
{Key: "duration", Value: "3600"},
},
})
defer s.Cluster.DataCoord.GcControl(ctx, &datapb.GcControlRequest{
Base: commonpbutil.NewMsgBase(),
Command: datapb.GcCommand_Resume,
})
// init collection with 2 channel, each channel has 2 segment, each segment has 2000 row
// and load it with 2 replicas on 2 nodes.
// then we add 2 query node, after balance happens, expected each node have 1 channel and 2 segments
name := "test_balance_" + funcutil.GenRandomStr()
s.initCollection(name, 2, 2, 2, 2000)
resp, err := s.Cluster.Proxy.GetReplicas(ctx, &milvuspb.GetReplicasRequest{CollectionName: name})
s.NoError(err)
s.Len(resp.Replicas, 2)
// add a querynode, expected balance happens
qn1 := s.Cluster.AddQueryNode()
qn2 := s.Cluster.AddQueryNode()
// check segment num on new query node
s.Eventually(func() bool {
resp, err := qn1.GetDataDistribution(ctx, &querypb.GetDataDistributionRequest{})
s.NoError(err)
return len(resp.Channels) == 1 && len(resp.Segments) == 2
}, 30*time.Second, 1*time.Second)
s.Eventually(func() bool {
resp, err := qn2.GetDataDistribution(ctx, &querypb.GetDataDistributionRequest{})
s.NoError(err)
return len(resp.Channels) == 1 && len(resp.Segments) == 2
}, 30*time.Second, 1*time.Second)
// check total segment num
s.Eventually(func() bool {
count := 0
for _, node := range s.Cluster.GetAllQueryNodes() {
resp1, err := node.GetDataDistribution(ctx, &querypb.GetDataDistributionRequest{})
s.NoError(err)
s.True(merr.Ok(resp1.GetStatus()))
count += len(resp1.Segments)
}
return count == 8
}, 10*time.Second, 1*time.Second)
}
func TestBalance(t *testing.T) {
suite.Run(t, new(BalanceTestSuit))
}

View File

@ -386,6 +386,13 @@ func (cluster *MiniClusterV2) Stop() error {
return nil
}
func (cluster *MiniClusterV2) GetAllQueryNodes() []*grpcquerynode.Server {
ret := make([]*grpcquerynode.Server, 0)
ret = append(ret, cluster.QueryNode)
ret = append(ret, cluster.querynodes...)
return ret
}
func (cluster *MiniClusterV2) StopAllQueryNodes() {
cluster.QueryNode.Stop()
log.Info("mini cluster main queryNode stopped")