mirror of https://github.com/milvus-io/milvus.git
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
parent
6a83f16871
commit
06df9b8462
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
}
|
|
@ -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")
|
||||
|
|
Loading…
Reference in New Issue