enhance: wal balance consider the wal status on streamingnode (#43265)

issue: #42995

- don't balance the wal if the producing-consuming lag is too long.
- don't balance if the rebalance is set as false.
- don't balance if the wal is balanced recently.

Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
Zhen Ye 2025-07-18 11:10:51 +08:00 committed by GitHub
parent 3aacd179f7
commit 07fa2cbdd3
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
26 changed files with 1490 additions and 787 deletions

View File

@ -1238,6 +1238,13 @@ streaming:
operationTimeout: 30s operationTimeout: 30s
balancePolicy: balancePolicy:
name: vchannelFair # The multiplier of balance task trigger backoff, 2 by default name: vchannelFair # The multiplier of balance task trigger backoff, 2 by default
# Whether to allow rebalance, true by default.
# If the rebalance is not allowed, only the lost wal recovery will be executed, the rebalance (move a pchannel from one node to another node) will be skipped.
allowRebalance: true
minRebalanceIntervalThreshold: 5m # The max interval of rebalance for each wal, 5m by default.
# The threshold of recovery lag for rebalance, 1s by default.
# If the recovery lag is greater than this threshold, the rebalance of current pchannel is not allowed.
allowRebalanceRecoveryLagThreshold: 1s
vchannelFair: vchannelFair:
# The weight of pchannel count in vchannelFair balance policy, # The weight of pchannel count in vchannelFair balance policy,
# the pchannel count will more evenly distributed if the weight is greater, 0.4 by default # the pchannel count will more evenly distributed if the weight is greater, 0.4 by default

View File

@ -436,14 +436,15 @@ func generateCurrentLayout(view *channel.PChannelView, allNodesStatus map[int64]
) )
} }
} }
allNodesInfo := make(map[int64]types.StreamingNodeInfo, len(allNodesStatus)) allNodesInfo := make(map[int64]types.StreamingNodeStatus, len(allNodesStatus))
for serverID, nodeStatus := range allNodesStatus { for serverID, nodeStatus := range allNodesStatus {
// filter out the unhealthy nodes. // filter out the unhealthy nodes.
if nodeStatus.IsHealthy() { if nodeStatus.IsHealthy() {
allNodesInfo[serverID] = nodeStatus.StreamingNodeInfo allNodesInfo[serverID] = *nodeStatus
} }
} }
return CurrentLayout{ return CurrentLayout{
Config: newCommonBalancePolicyConfig(),
Channels: channels, Channels: channels,
Stats: view.Stats, Stats: view.Stats,
AllNodesInfo: allNodesInfo, AllNodesInfo: allNodesInfo,

View File

@ -10,6 +10,7 @@ import (
"github.com/cockroachdb/errors" "github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock" "github.com/stretchr/testify/mock"
"go.uber.org/atomic"
"github.com/milvus-io/milvus/internal/mocks/mock_metastore" "github.com/milvus-io/milvus/internal/mocks/mock_metastore"
"github.com/milvus-io/milvus/internal/mocks/streamingnode/client/mock_manager" "github.com/milvus-io/milvus/internal/mocks/streamingnode/client/mock_manager"
@ -23,6 +24,7 @@ import (
"github.com/milvus-io/milvus/pkg/v2/util/etcd" "github.com/milvus-io/milvus/pkg/v2/util/etcd"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable" "github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/syncutil" "github.com/milvus-io/milvus/pkg/v2/util/syncutil"
"github.com/milvus-io/milvus/pkg/v2/util/tsoutil"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil" "github.com/milvus-io/milvus/pkg/v2/util/typeutil"
) )
@ -62,7 +64,7 @@ func TestBalancer(t *testing.T) {
}, },
4: { 4: {
StreamingNodeInfo: types.StreamingNodeInfo{ StreamingNodeInfo: types.StreamingNodeInfo{
ServerID: 3, ServerID: 4,
Address: "localhost:3", Address: "localhost:3",
}, },
Err: types.ErrStopping, Err: types.ErrStopping,
@ -185,3 +187,132 @@ func TestBalancer(t *testing.T) {
b.Close() b.Close()
assert.ErrorIs(t, f.Get(), balancer.ErrBalancerClosed) assert.ErrorIs(t, f.Get(), balancer.ErrBalancerClosed)
} }
func TestBalancer_WithRecoveryLag(t *testing.T) {
paramtable.Init()
err := etcd.InitEtcdServer(true, "", t.TempDir(), "stdout", "info")
assert.NoError(t, err)
defer etcd.StopEtcdServer()
etcdClient, err := etcd.GetEmbedEtcdClient()
assert.NoError(t, err)
channel.ResetStaticPChannelStatsManager()
channel.RecoverPChannelStatsManager([]string{})
lag := atomic.NewBool(true)
streamingNodeManager := mock_manager.NewMockManagerClient(t)
streamingNodeManager.EXPECT().WatchNodeChanged(mock.Anything).Return(make(chan struct{}), nil)
streamingNodeManager.EXPECT().Assign(mock.Anything, mock.Anything).Return(nil)
streamingNodeManager.EXPECT().Remove(mock.Anything, mock.Anything).Return(nil)
streamingNodeManager.EXPECT().CollectAllStatus(mock.Anything).RunAndReturn(func(ctx context.Context) (map[int64]*types.StreamingNodeStatus, error) {
now := time.Now()
mvccTimeTick := tsoutil.ComposeTSByTime(now, 0)
recoveryTimeTick := tsoutil.ComposeTSByTime(now.Add(-time.Second*10), 0)
if !lag.Load() {
recoveryTimeTick = mvccTimeTick
}
return map[int64]*types.StreamingNodeStatus{
1: {
StreamingNodeInfo: types.StreamingNodeInfo{
ServerID: 1,
Address: "localhost:1",
},
Metrics: types.StreamingNodeMetrics{
WALMetrics: map[types.ChannelID]types.WALMetrics{
channel.ChannelID{Name: "test-channel-1"}: types.RWWALMetrics{MVCCTimeTick: mvccTimeTick, RecoveryTimeTick: recoveryTimeTick},
channel.ChannelID{Name: "test-channel-2"}: types.RWWALMetrics{MVCCTimeTick: mvccTimeTick, RecoveryTimeTick: recoveryTimeTick},
channel.ChannelID{Name: "test-channel-3"}: types.RWWALMetrics{MVCCTimeTick: mvccTimeTick, RecoveryTimeTick: recoveryTimeTick},
},
},
},
2: {
StreamingNodeInfo: types.StreamingNodeInfo{
ServerID: 2,
Address: "localhost:2",
},
},
}, nil
})
catalog := mock_metastore.NewMockStreamingCoordCataLog(t)
resource.InitForTest(resource.OptETCD(etcdClient), resource.OptStreamingCatalog(catalog), resource.OptStreamingManagerClient(streamingNodeManager))
catalog.EXPECT().GetVersion(mock.Anything).Return(nil, nil)
catalog.EXPECT().SaveVersion(mock.Anything, mock.Anything).Return(nil)
catalog.EXPECT().ListPChannel(mock.Anything).Unset()
catalog.EXPECT().ListPChannel(mock.Anything).RunAndReturn(func(ctx context.Context) ([]*streamingpb.PChannelMeta, error) {
return []*streamingpb.PChannelMeta{
{
Channel: &streamingpb.PChannelInfo{
Name: "test-channel-1",
Term: 1,
AccessMode: streamingpb.PChannelAccessMode_PCHANNEL_ACCESS_READWRITE,
},
State: streamingpb.PChannelMetaState_PCHANNEL_META_STATE_ASSIGNED,
Node: &streamingpb.StreamingNodeInfo{ServerId: 1},
},
{
Channel: &streamingpb.PChannelInfo{
Name: "test-channel-2",
Term: 1,
AccessMode: streamingpb.PChannelAccessMode_PCHANNEL_ACCESS_READWRITE,
},
State: streamingpb.PChannelMetaState_PCHANNEL_META_STATE_ASSIGNED,
Node: &streamingpb.StreamingNodeInfo{ServerId: 1},
},
{
Channel: &streamingpb.PChannelInfo{
Name: "test-channel-3",
Term: 1,
AccessMode: streamingpb.PChannelAccessMode_PCHANNEL_ACCESS_READWRITE,
},
State: streamingpb.PChannelMetaState_PCHANNEL_META_STATE_ASSIGNED,
Node: &streamingpb.StreamingNodeInfo{ServerId: 1},
},
{
Channel: &streamingpb.PChannelInfo{
Name: "test-channel-4",
Term: 1,
AccessMode: streamingpb.PChannelAccessMode_PCHANNEL_ACCESS_READWRITE,
},
State: streamingpb.PChannelMetaState_PCHANNEL_META_STATE_ASSIGNED,
Node: &streamingpb.StreamingNodeInfo{ServerId: 2},
},
}, nil
})
catalog.EXPECT().SavePChannels(mock.Anything, mock.Anything).Return(nil).Maybe()
ctx := context.Background()
b, err := balancer.RecoverBalancer(ctx)
assert.NoError(t, err)
assert.NotNil(t, b)
b.Trigger(context.Background())
ctx2, cancel := context.WithTimeout(ctx, 2*time.Second)
defer cancel()
b.WatchChannelAssignments(ctx2, func(version typeutil.VersionInt64Pair, relations []types.PChannelInfoAssigned) error {
counts := map[int64]int{}
for _, relation := range relations {
assert.Equal(t, relation.Channel.AccessMode, types.AccessModeRW)
counts[relation.Node.ServerID]++
}
assert.Equal(t, 2, len(counts))
assert.Equal(t, 3, counts[1])
assert.Equal(t, 1, counts[2])
return nil
})
lag.Store(false)
b.Trigger(context.Background())
doneErr := errors.New("done")
b.WatchChannelAssignments(context.Background(), func(version typeutil.VersionInt64Pair, relations []types.PChannelInfoAssigned) error {
counts := map[int64]int{}
for _, relation := range relations {
assert.Equal(t, relation.Channel.AccessMode, types.AccessModeRW)
counts[relation.Node.ServerID]++
}
if len(counts) == 2 && counts[1] == 2 && counts[2] == 2 {
return doneErr
}
return nil
})
}

View File

@ -1,6 +1,8 @@
package channel package channel
import ( import (
"time"
"google.golang.org/protobuf/proto" "google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb" "github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
@ -91,6 +93,11 @@ func (c *PChannelMeta) IsAssigned() bool {
return c.inner.State == streamingpb.PChannelMetaState_PCHANNEL_META_STATE_ASSIGNED return c.inner.State == streamingpb.PChannelMetaState_PCHANNEL_META_STATE_ASSIGNED
} }
// LastAssignTimestamp returns the last assigned timestamp.
func (c *PChannelMeta) LastAssignTimestamp() time.Time {
return time.Unix(int64(c.inner.LastAssignTimestampSeconds), 0)
}
// State returns the state of the channel. // State returns the state of the channel.
func (c *PChannelMeta) State() streamingpb.PChannelMetaState { func (c *PChannelMeta) State() streamingpb.PChannelMetaState {
return c.inner.State return c.inner.State
@ -140,6 +147,7 @@ func (m *mutablePChannel) AssignToServerDone() {
if m.inner.State == streamingpb.PChannelMetaState_PCHANNEL_META_STATE_ASSIGNING { if m.inner.State == streamingpb.PChannelMetaState_PCHANNEL_META_STATE_ASSIGNING {
m.inner.Histories = make([]*streamingpb.PChannelAssignmentLog, 0) m.inner.Histories = make([]*streamingpb.PChannelAssignmentLog, 0)
m.inner.State = streamingpb.PChannelMetaState_PCHANNEL_META_STATE_ASSIGNED m.inner.State = streamingpb.PChannelMetaState_PCHANNEL_META_STATE_ASSIGNED
m.inner.LastAssignTimestampSeconds = uint64(time.Now().Unix())
} }
} }

View File

@ -2,6 +2,7 @@ package channel
import ( import (
"fmt" "fmt"
"time"
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types" "github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
) )
@ -21,7 +22,9 @@ func newPChannelView(metas map[ChannelID]*PChannelMeta) *PChannelView {
panic(fmt.Sprintf("duplicate rw channel: %s", id.String())) panic(fmt.Sprintf("duplicate rw channel: %s", id.String()))
} }
view.Channels[id] = meta view.Channels[id] = meta
view.Stats[id] = StaticPChannelStatsManager.MustGet().GetPChannelStats(id).View() stat := StaticPChannelStatsManager.MustGet().GetPChannelStats(id).View()
stat.LastAssignTimestamp = meta.LastAssignTimestamp()
view.Stats[id] = stat
} }
return view return view
} }
@ -34,5 +37,6 @@ type PChannelView struct {
// PChannelStatsView is the view of the pchannel stats. // PChannelStatsView is the view of the pchannel stats.
type PChannelStatsView struct { type PChannelStatsView struct {
LastAssignTimestamp time.Time
VChannels map[string]int64 VChannels map[string]int64
} }

View File

@ -108,7 +108,7 @@ func (p *expectedLayoutForVChannelFairPolicy) Assign(channelID types.ChannelID,
info.Term++ info.Term++
p.Assignments[channelID] = types.PChannelInfoAssigned{ p.Assignments[channelID] = types.PChannelInfoAssigned{
Channel: info, Channel: info,
Node: node, Node: node.StreamingNodeInfo,
} }
p.Nodes[node.ServerID].AssignedChannels[channelID] = struct{}{} p.Nodes[node.ServerID].AssignedChannels[channelID] = struct{}{}
p.Nodes[node.ServerID].AssignedVChannelCount += len(stats.VChannels) p.Nodes[node.ServerID].AssignedVChannelCount += len(stats.VChannels)
@ -171,6 +171,9 @@ func (p *expectedLayoutForVChannelFairPolicy) FindTheLeastUnbalanceScoreIncremen
var targetChannelID types.ChannelID var targetChannelID types.ChannelID
minScore := math.MaxFloat64 minScore := math.MaxFloat64
for channelID := range p.Assignments { for channelID := range p.Assignments {
if !p.CurrentLayout.AllowRebalance(channelID) {
continue
}
serverID := p.Assignments[channelID].Node.ServerID serverID := p.Assignments[channelID].Node.ServerID
p.Unassign(channelID) p.Unassign(channelID)
currentScore := p.GlobalUnbalancedScore currentScore := p.GlobalUnbalancedScore

View File

@ -2,6 +2,7 @@ package vchannelfair
import ( import (
"testing" "testing"
"time"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
@ -101,10 +102,107 @@ func TestPChannelCountFair(t *testing.T) {
} }
assert.NoError(t, err) assert.NoError(t, err)
expected, err = policy.Balance(newLayout(map[string]int{
"c1": 1,
"c2": 2,
"c3": 3,
"c4": 3,
"c5": 3,
}, make(map[string]map[string]int64), []int64{1, 2, 3}))
assert.NoError(t, err)
_, err = policy.Balance(balancer.CurrentLayout{}) _, err = policy.Balance(balancer.CurrentLayout{})
assert.Error(t, err) assert.Error(t, err)
} }
func TestPChannelCountFairWithoutRebalance(t *testing.T) {
paramtable.Init()
b := &PolicyBuilder{}
policy := b.Build()
createNewLayout := func() balancer.CurrentLayout {
return newLayout(map[string]int{
"c1": 1,
"c2": 2,
"c3": 3,
"c4": 3,
"c5": 3,
"c6": 3,
}, make(map[string]map[string]int64), []int64{1, 2, 3})
}
// test allow rebalance
layout := createNewLayout()
expected, err := policy.Balance(layout)
assert.NoError(t, err)
assert.Equal(t, 6, len(expected.ChannelAssignment))
counts := countByServerID(expected)
assert.Equal(t, 3, len(counts))
for _, count := range counts {
assert.Equal(t, count, 2)
}
// test allow rebalance = false
layout = createNewLayout()
layout.Config.AllowRebalance = false
expected, err = policy.Balance(layout)
assert.NoError(t, err)
assert.Equal(t, 6, len(expected.ChannelAssignment))
assert.Equal(t, int64(1), expected.ChannelAssignment[newChannelID("c1")].Node.ServerID)
assert.Equal(t, int64(2), expected.ChannelAssignment[newChannelID("c2")].Node.ServerID)
assert.Equal(t, int64(3), expected.ChannelAssignment[newChannelID("c3")].Node.ServerID)
assert.Equal(t, int64(3), expected.ChannelAssignment[newChannelID("c4")].Node.ServerID)
assert.Equal(t, int64(3), expected.ChannelAssignment[newChannelID("c5")].Node.ServerID)
assert.Equal(t, int64(3), expected.ChannelAssignment[newChannelID("c6")].Node.ServerID)
assert.Equal(t, 3, len(countByServerID(expected)))
// test min rebalance interval threshold
layout = createNewLayout()
layout.Config.MinRebalanceIntervalThreshold = time.Hour
layout.Stats[newChannelID("c3")] = channel.PChannelStatsView{
LastAssignTimestamp: time.Now(),
VChannels: map[string]int64{},
}
layout.Stats[newChannelID("c4")] = channel.PChannelStatsView{
LastAssignTimestamp: time.Now(),
VChannels: map[string]int64{},
}
layout.Stats[newChannelID("c5")] = channel.PChannelStatsView{
LastAssignTimestamp: time.Now(),
VChannels: map[string]int64{},
}
expected, err = policy.Balance(layout)
assert.NoError(t, err)
assert.Equal(t, 6, len(expected.ChannelAssignment))
assert.Equal(t, int64(3), expected.ChannelAssignment[newChannelID("c3")].Node.ServerID)
assert.Equal(t, int64(3), expected.ChannelAssignment[newChannelID("c4")].Node.ServerID)
assert.Equal(t, int64(3), expected.ChannelAssignment[newChannelID("c5")].Node.ServerID)
assert.NotEqual(t, int64(3), expected.ChannelAssignment[newChannelID("c6")].Node.ServerID)
assert.Equal(t, 3, len(countByServerID(expected)))
// test allow rebalance = false, but there's some channel is not assigned.
layout = newLayout(map[string]int{
"c1": -1,
"c2": 3,
"c3": -1,
"c4": -1,
"c5": 3,
"c6": 3,
}, make(map[string]map[string]int64), []int64{1, 2, 3})
layout.Config.AllowRebalance = false
expected, err = policy.Balance(layout)
assert.NoError(t, err)
assert.Equal(t, 6, len(expected.ChannelAssignment))
assert.Equal(t, int64(3), expected.ChannelAssignment[newChannelID("c2")].Node.ServerID)
assert.Equal(t, int64(3), expected.ChannelAssignment[newChannelID("c5")].Node.ServerID)
assert.Equal(t, int64(3), expected.ChannelAssignment[newChannelID("c6")].Node.ServerID)
assert.NotEqual(t, int64(3), expected.ChannelAssignment[newChannelID("c1")].Node.ServerID)
assert.NotEqual(t, int64(3), expected.ChannelAssignment[newChannelID("c3")].Node.ServerID)
assert.NotEqual(t, int64(3), expected.ChannelAssignment[newChannelID("c4")].Node.ServerID)
assert.Equal(t, 3, len(countByServerID(expected)))
}
func countByServerID(expected balancer.ExpectedLayout) map[int64]int { func countByServerID(expected balancer.ExpectedLayout) map[int64]int {
counts := make(map[int64]int) counts := make(map[int64]int)
for _, node := range expected.ChannelAssignment { for _, node := range expected.ChannelAssignment {
@ -122,15 +220,22 @@ func newChannelID(channel string) types.ChannelID {
// newLayout creates a new layout for test. // newLayout creates a new layout for test.
func newLayout(channels map[string]int, vchannels map[string]map[string]int64, serverID []int64) balancer.CurrentLayout { func newLayout(channels map[string]int, vchannels map[string]map[string]int64, serverID []int64) balancer.CurrentLayout {
layout := balancer.CurrentLayout{ layout := balancer.CurrentLayout{
Config: balancer.CommonBalancePolicyConfig{
AllowRebalance: true,
AllowRebalanceRecoveryLagThreshold: 1 * time.Second,
MinRebalanceIntervalThreshold: 1 * time.Second,
},
Channels: make(map[channel.ChannelID]types.PChannelInfo), Channels: make(map[channel.ChannelID]types.PChannelInfo),
Stats: make(map[channel.ChannelID]channel.PChannelStatsView), Stats: make(map[channel.ChannelID]channel.PChannelStatsView),
AllNodesInfo: make(map[int64]types.StreamingNodeInfo), AllNodesInfo: make(map[int64]types.StreamingNodeStatus),
ChannelsToNodes: make(map[types.ChannelID]int64), ChannelsToNodes: make(map[types.ChannelID]int64),
ExpectedAccessMode: make(map[channel.ChannelID]types.AccessMode), ExpectedAccessMode: make(map[channel.ChannelID]types.AccessMode),
} }
for _, id := range serverID { for _, id := range serverID {
layout.AllNodesInfo[id] = types.StreamingNodeInfo{ layout.AllNodesInfo[id] = types.StreamingNodeStatus{
StreamingNodeInfo: types.StreamingNodeInfo{
ServerID: id, ServerID: id,
},
} }
} }
for c, node := range channels { for c, node := range channels {

View File

@ -70,14 +70,24 @@ func (p *policy) Balance(currentLayout balancer.CurrentLayout) (layout balancer.
} }
} }
// 3. Unassign some most unbalanced channel and reassign it, try to make the layout more balanced
snapshot := expectedLayout.AssignmentSnapshot() snapshot := expectedLayout.AssignmentSnapshot()
if !currentLayout.Config.AllowRebalance {
return balancer.ExpectedLayout{
ChannelAssignment: snapshot.Assignments,
}, nil
}
// The following code is the operation for the auto balance
// 3. Unassign some most unbalanced channel and reassign it, try to make the layout more balanced
reassignChannelIDs := make([]types.ChannelID, 0, p.cfg.RebalanceMaxStep) reassignChannelIDs := make([]types.ChannelID, 0, p.cfg.RebalanceMaxStep)
for i := 0; i < p.cfg.RebalanceMaxStep; i++ { for i := 0; i < p.cfg.RebalanceMaxStep; i++ {
if len(expectedLayout.Assignments) == 0 { if len(expectedLayout.Assignments) == 0 {
break break
} }
channelID := expectedLayout.FindTheLeastUnbalanceScoreIncrementChannel() channelID := expectedLayout.FindTheLeastUnbalanceScoreIncrementChannel()
if channelID.IsZero() {
break
}
expectedLayout.Unassign(channelID) expectedLayout.Unassign(channelID)
reassignChannelIDs = append(reassignChannelIDs, channelID) reassignChannelIDs = append(reassignChannelIDs, channelID)
} }

View File

@ -3,23 +3,43 @@ package balancer
import ( import (
"sort" "sort"
"strings" "strings"
"time"
"github.com/samber/lo" "github.com/samber/lo"
"github.com/milvus-io/milvus/internal/streamingcoord/server/balancer/channel" "github.com/milvus-io/milvus/internal/streamingcoord/server/balancer/channel"
"github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/streaming/util/types" "github.com/milvus-io/milvus/pkg/v2/streaming/util/types"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil" "github.com/milvus-io/milvus/pkg/v2/util/typeutil"
) )
// policiesBuilders is a map of registered balancer policiesBuilders. // policiesBuilders is a map of registered balancer policiesBuilders.
var policiesBuilders typeutil.ConcurrentMap[string, PolicyBuilder] var policiesBuilders typeutil.ConcurrentMap[string, PolicyBuilder]
// newCommonBalancePolicyConfig returns the common balance policy config.
func newCommonBalancePolicyConfig() CommonBalancePolicyConfig {
params := paramtable.Get()
return CommonBalancePolicyConfig{
AllowRebalance: params.StreamingCfg.WALBalancerPolicyAllowRebalance.GetAsBool(),
AllowRebalanceRecoveryLagThreshold: params.StreamingCfg.WALBalancerPolicyAllowRebalanceRecoveryLagThreshold.GetAsDurationByParse(),
MinRebalanceIntervalThreshold: params.StreamingCfg.WALBalancerPolicyMinRebalanceIntervalThreshold.GetAsDurationByParse(),
}
}
// CommonBalancePolicyConfig is the config for balance policy.
type CommonBalancePolicyConfig struct {
AllowRebalance bool // Whether to allow rebalance.
AllowRebalanceRecoveryLagThreshold time.Duration // The threshold of recovery lag for balance.
MinRebalanceIntervalThreshold time.Duration // The min interval of rebalance.
}
// CurrentLayout is the full topology of streaming node and pChannel. // CurrentLayout is the full topology of streaming node and pChannel.
type CurrentLayout struct { type CurrentLayout struct {
Config CommonBalancePolicyConfig
Channels map[channel.ChannelID]types.PChannelInfo Channels map[channel.ChannelID]types.PChannelInfo
Stats map[channel.ChannelID]channel.PChannelStatsView Stats map[channel.ChannelID]channel.PChannelStatsView
AllNodesInfo map[int64]types.StreamingNodeInfo // AllNodesInfo is the full information of all available streaming nodes and related pchannels (contain the node not assign anything on it). AllNodesInfo map[int64]types.StreamingNodeStatus // AllNodesInfo is the full information of all available streaming nodes and related pchannels (contain the node not assign anything on it).
ChannelsToNodes map[types.ChannelID]int64 // ChannelsToNodes maps assigned channel name to node id. ChannelsToNodes map[types.ChannelID]int64 // ChannelsToNodes maps assigned channel name to node id.
ExpectedAccessMode map[channel.ChannelID]types.AccessMode // ExpectedAccessMode is the expected access mode of all channel. ExpectedAccessMode map[channel.ChannelID]types.AccessMode // ExpectedAccessMode is the expected access mode of all channel.
} }
@ -54,6 +74,44 @@ func (layout *CurrentLayout) TotalNodes() int {
return len(layout.AllNodesInfo) return len(layout.AllNodesInfo)
} }
// AllowRebalance returns true if the balance of the pchannel is allowed.
func (layout *CurrentLayout) AllowRebalance(channelID channel.ChannelID) bool {
if !layout.Config.AllowRebalance {
// If rebalance is not allowed, return false directly.
return false
}
// If the last assign timestamp is too close to the current time, rebalance is not allowed.
if time.Since(layout.Stats[channelID].LastAssignTimestamp) < layout.Config.MinRebalanceIntervalThreshold {
return false
}
// If reach the recovery lag threshold, rebalance is not allowed.
return !layout.isReachTheRecoveryLagThreshold(channelID)
}
// isReachTheRecoveryLagThreshold returns true if the recovery lag of the pchannel is greater than the recovery lag threshold.
func (layout *CurrentLayout) isReachTheRecoveryLagThreshold(channelID channel.ChannelID) bool {
balanceAttr := layout.GetWALMetrics(channelID)
if balanceAttr == nil {
return false
}
r, ok := balanceAttr.(types.RWWALMetrics)
if !ok {
return false
}
return r.RecoveryLag() > layout.Config.AllowRebalanceRecoveryLagThreshold
}
// GetWALMetrics returns the WAL metrics of the pchannel.
func (layout *CurrentLayout) GetWALMetrics(channelID channel.ChannelID) types.WALMetrics {
node, ok := layout.ChannelsToNodes[channelID]
if !ok {
return nil
}
return layout.AllNodesInfo[node].Metrics.WALMetrics[channelID]
}
// GetAllPChannelsSortedByVChannelCountDesc returns all pchannels sorted by vchannel count in descending order. // GetAllPChannelsSortedByVChannelCountDesc returns all pchannels sorted by vchannel count in descending order.
func (layout *CurrentLayout) GetAllPChannelsSortedByVChannelCountDesc() []types.ChannelID { func (layout *CurrentLayout) GetAllPChannelsSortedByVChannelCountDesc() []types.ChannelID {
sorter := make(byVChannelCountDesc, 0, layout.TotalChannels()) sorter := make(byVChannelCountDesc, 0, layout.TotalChannels())

View File

@ -102,7 +102,7 @@ func (c *managerClientImpl) getAllStreamingNodeStatus(ctx context.Context, state
} }
g, _ := errgroup.WithContext(ctx) g, _ := errgroup.WithContext(ctx)
g.SetLimit(10) g.SetLimit(16)
var mu sync.Mutex var mu sync.Mutex
result := make(map[int64]*types.StreamingNodeStatus, len(state.State.Addresses)) result := make(map[int64]*types.StreamingNodeStatus, len(state.State.Addresses))
for serverID, session := range state.Sessions() { for serverID, session := range state.Sessions() {
@ -113,24 +113,24 @@ func (c *managerClientImpl) getAllStreamingNodeStatus(ctx context.Context, state
resp, err := manager.CollectStatus(ctx, &streamingpb.StreamingNodeManagerCollectStatusRequest{}) resp, err := manager.CollectStatus(ctx, &streamingpb.StreamingNodeManagerCollectStatusRequest{})
mu.Lock() mu.Lock()
defer mu.Unlock() defer mu.Unlock()
if err != nil {
log.Warn("collect status failed, skip", zap.Int64("serverID", serverID), zap.Error(err))
return err
}
result[serverID] = &types.StreamingNodeStatus{ result[serverID] = &types.StreamingNodeStatus{
StreamingNodeInfo: types.StreamingNodeInfo{ StreamingNodeInfo: types.StreamingNodeInfo{
ServerID: serverID, ServerID: serverID,
Address: address, Address: address,
}, },
Metrics: types.NewStreamingNodeBalanceAttrsFromProto(resp.Metrics),
Err: err, Err: err,
} }
if err != nil {
log.Warn("collect status failed, skip", zap.Int64("serverID", serverID), zap.Error(err))
return err
}
log.Debug("collect status success", zap.Int64("serverID", serverID), zap.Any("status", resp)) log.Debug("collect status success", zap.Int64("serverID", serverID), zap.Any("status", resp))
return nil return nil
}) })
} }
g.Wait() g.Wait()
return result, nil return result, nil
} }

View File

@ -50,8 +50,11 @@ func (ms *managerServiceImpl) Remove(ctx context.Context, req *streamingpb.Strea
// CollectStatus collects the status of all wal instances in these streamingnode. // CollectStatus collects the status of all wal instances in these streamingnode.
func (ms *managerServiceImpl) CollectStatus(ctx context.Context, req *streamingpb.StreamingNodeManagerCollectStatusRequest) (*streamingpb.StreamingNodeManagerCollectStatusResponse, error) { func (ms *managerServiceImpl) CollectStatus(ctx context.Context, req *streamingpb.StreamingNodeManagerCollectStatusRequest) (*streamingpb.StreamingNodeManagerCollectStatusResponse, error) {
// TODO: collect traffic metric for load balance. metrics, err := ms.walManager.Metrics()
if err != nil {
return nil, err
}
return &streamingpb.StreamingNodeManagerCollectStatusResponse{ return &streamingpb.StreamingNodeManagerCollectStatusResponse{
BalanceAttributes: &streamingpb.StreamingNodeBalanceAttributes{}, Metrics: types.NewProtoFromStreamingNodeMetrics(*metrics),
}, nil }, nil
} }

View File

@ -133,26 +133,6 @@ func (m *managerImpl) Metrics() (*types.StreamingNodeMetrics, error) {
}, nil }, nil
} }
// GetAllAvailableChannels returns all available channel info.
func (m *managerImpl) GetAllAvailableChannels() ([]types.PChannelInfo, error) {
// reject operation if manager is closing.
if !m.lifetime.AddIf(isGetable) {
return nil, errWALManagerClosed
}
defer m.lifetime.Done()
// collect all available wal info.
infos := make([]types.PChannelInfo, 0)
m.wltMap.Range(func(channel string, lt *walLifetime) bool {
if l := lt.GetWAL(); l != nil {
info := l.Channel()
infos = append(infos, info)
}
return true
})
return infos, nil
}
// Close these manager and release all managed WAL. // Close these manager and release all managed WAL.
func (m *managerImpl) Close() { func (m *managerImpl) Close() {
m.lifetime.SetState(managerRemoveable) m.lifetime.SetState(managerRemoveable)

View File

@ -42,6 +42,7 @@ func TestManager(t *testing.T) {
l.EXPECT().IsAvailable().Return(true).Maybe() l.EXPECT().IsAvailable().Return(true).Maybe()
l.EXPECT().Close().Return() l.EXPECT().Close().Return()
l.EXPECT().IsAvailable().Return(true).Maybe() l.EXPECT().IsAvailable().Return(true).Maybe()
l.EXPECT().Metrics().Return(types.RWWALMetrics{})
return l, nil return l, nil
}) })
opener.EXPECT().Close().Return() opener.EXPECT().Close().Return()

View File

@ -60,6 +60,7 @@ message PChannelMeta {
PChannelMetaState state = 3; // state of the channel. PChannelMetaState state = 3; // state of the channel.
repeated PChannelAssignmentLog histories = repeated PChannelAssignmentLog histories =
4; // keep the meta info assignment log that used to be assigned to. 4; // keep the meta info assignment log that used to be assigned to.
uint64 last_assign_timestamp_seconds = 5; // The last assigned timestamp in seconds.
} }
// StreamingVersion is the version of the streaming service. // StreamingVersion is the version of the streaming service.
@ -486,12 +487,28 @@ message StreamingNodeManagerRemoveResponse {}
message StreamingNodeManagerCollectStatusRequest {} message StreamingNodeManagerCollectStatusRequest {}
message StreamingNodeBalanceAttributes { message StreamingNodeMetrics {
// TODO: traffic of pchannel or other things. repeated StreamingNodeWALMetrics wals = 1;
}
message StreamingNodeWALMetrics {
PChannelInfo info = 1;
oneof metrics {
StreamingNodeRWWALMetrics rw = 2;
StreamingNodeROWALMetrics ro = 3;
};
}
message StreamingNodeRWWALMetrics {
uint64 mvcc_time_tick = 1; // The mvcc time tick of the pchannel.
uint64 recovery_time_tick = 2; // The recovery time tick of the pchannel.
}
message StreamingNodeROWALMetrics {
} }
message StreamingNodeManagerCollectStatusResponse { message StreamingNodeManagerCollectStatusResponse {
StreamingNodeBalanceAttributes balance_attributes = 1; StreamingNodeMetrics metrics = 1;
} }
/// ///

File diff suppressed because it is too large Load Diff

View File

@ -74,7 +74,7 @@ func (n StreamingNodeInfo) String() string {
// StreamingNodeStatus is the information of a streaming node. // StreamingNodeStatus is the information of a streaming node.
type StreamingNodeStatus struct { type StreamingNodeStatus struct {
StreamingNodeInfo StreamingNodeInfo
// TODO: balance attributes should added here in future. Metrics StreamingNodeMetrics
Err error Err error
} }

View File

@ -3,6 +3,7 @@ package types
import ( import (
"time" "time"
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/v2/util/tsoutil" "github.com/milvus-io/milvus/pkg/v2/util/tsoutil"
) )
@ -44,3 +45,53 @@ func (ROWALMetrics) isWALMetrics() {}
type StreamingNodeMetrics struct { type StreamingNodeMetrics struct {
WALMetrics map[ChannelID]WALMetrics WALMetrics map[ChannelID]WALMetrics
} }
func NewStreamingNodeBalanceAttrsFromProto(m *streamingpb.StreamingNodeMetrics) StreamingNodeMetrics {
channels := make(map[ChannelID]WALMetrics)
for _, l := range m.GetWals() {
switch balanceAttr := l.Metrics.(type) {
case *streamingpb.StreamingNodeWALMetrics_Ro:
channels[ChannelID{Name: l.Info.Name}] = ROWALMetrics{
ChannelInfo: NewPChannelInfoFromProto(l.Info),
}
case *streamingpb.StreamingNodeWALMetrics_Rw:
channels[ChannelID{Name: l.Info.Name}] = RWWALMetrics{
ChannelInfo: NewPChannelInfoFromProto(l.Info),
MVCCTimeTick: balanceAttr.Rw.MvccTimeTick,
RecoveryTimeTick: balanceAttr.Rw.RecoveryTimeTick,
}
}
}
return StreamingNodeMetrics{
WALMetrics: channels,
}
}
// NewProtoFrom
func NewProtoFromStreamingNodeMetrics(info StreamingNodeMetrics) *streamingpb.StreamingNodeMetrics {
wals := make([]*streamingpb.StreamingNodeWALMetrics, 0, len(info.WALMetrics))
for _, metrics := range info.WALMetrics {
switch metrics := metrics.(type) {
case RWWALMetrics:
wals = append(wals, &streamingpb.StreamingNodeWALMetrics{
Info: NewProtoFromPChannelInfo(metrics.ChannelInfo),
Metrics: &streamingpb.StreamingNodeWALMetrics_Rw{
Rw: &streamingpb.StreamingNodeRWWALMetrics{
MvccTimeTick: metrics.MVCCTimeTick,
RecoveryTimeTick: metrics.RecoveryTimeTick,
},
},
})
case ROWALMetrics:
wals = append(wals, &streamingpb.StreamingNodeWALMetrics{
Info: NewProtoFromPChannelInfo(metrics.ChannelInfo),
Metrics: &streamingpb.StreamingNodeWALMetrics_Ro{
Ro: &streamingpb.StreamingNodeROWALMetrics{},
},
})
}
}
return &streamingpb.StreamingNodeMetrics{
Wals: wals,
}
}

View File

@ -5724,6 +5724,9 @@ type streamingConfig struct {
// balancer Policy // balancer Policy
WALBalancerPolicyName ParamItem `refreshable:"true"` WALBalancerPolicyName ParamItem `refreshable:"true"`
WALBalancerPolicyAllowRebalance ParamItem `refreshable:"true"`
WALBalancerPolicyMinRebalanceIntervalThreshold ParamItem `refreshable:"true"`
WALBalancerPolicyAllowRebalanceRecoveryLagThreshold ParamItem `refreshable:"true"`
WALBalancerPolicyVChannelFairPChannelWeight ParamItem `refreshable:"true"` WALBalancerPolicyVChannelFairPChannelWeight ParamItem `refreshable:"true"`
WALBalancerPolicyVChannelFairVChannelWeight ParamItem `refreshable:"true"` WALBalancerPolicyVChannelFairVChannelWeight ParamItem `refreshable:"true"`
WALBalancerPolicyVChannelFairAntiAffinityWeight ParamItem `refreshable:"true"` WALBalancerPolicyVChannelFairAntiAffinityWeight ParamItem `refreshable:"true"`
@ -5815,6 +5818,35 @@ If the operation exceeds this timeout, it will be canceled.`,
} }
p.WALBalancerPolicyName.Init(base.mgr) p.WALBalancerPolicyName.Init(base.mgr)
p.WALBalancerPolicyAllowRebalance = ParamItem{
Key: "streaming.walBalancer.balancePolicy.allowRebalance",
Version: "2.6.0",
Doc: `Whether to allow rebalance, true by default.
If the rebalance is not allowed, only the lost wal recovery will be executed, the rebalance (move a pchannel from one node to another node) will be skipped.`,
DefaultValue: "true",
Export: true,
}
p.WALBalancerPolicyAllowRebalance.Init(base.mgr)
p.WALBalancerPolicyMinRebalanceIntervalThreshold = ParamItem{
Key: "streaming.walBalancer.balancePolicy.minRebalanceIntervalThreshold",
Version: "2.6.0",
Doc: `The max interval of rebalance for each wal, 5m by default.`,
DefaultValue: "5m",
Export: true,
}
p.WALBalancerPolicyMinRebalanceIntervalThreshold.Init(base.mgr)
p.WALBalancerPolicyAllowRebalanceRecoveryLagThreshold = ParamItem{
Key: "streaming.walBalancer.balancePolicy.allowRebalanceRecoveryLagThreshold",
Version: "2.6.0",
Doc: `The threshold of recovery lag for rebalance, 1s by default.
If the recovery lag is greater than this threshold, the rebalance of current pchannel is not allowed.`,
DefaultValue: "1s",
Export: true,
}
p.WALBalancerPolicyAllowRebalanceRecoveryLagThreshold.Init(base.mgr)
p.WALBalancerPolicyVChannelFairPChannelWeight = ParamItem{ p.WALBalancerPolicyVChannelFairPChannelWeight = ParamItem{
Key: "streaming.walBalancer.balancePolicy.vchannelFair.pchannelWeight", Key: "streaming.walBalancer.balancePolicy.vchannelFair.pchannelWeight",
Version: "2.6.0", Version: "2.6.0",

View File

@ -646,6 +646,9 @@ func TestComponentParam(t *testing.T) {
assert.Equal(t, 5*time.Second, params.StreamingCfg.WALBalancerBackoffMaxInterval.GetAsDurationByParse()) assert.Equal(t, 5*time.Second, params.StreamingCfg.WALBalancerBackoffMaxInterval.GetAsDurationByParse())
assert.Equal(t, 2.0, params.StreamingCfg.WALBalancerBackoffMultiplier.GetAsFloat()) assert.Equal(t, 2.0, params.StreamingCfg.WALBalancerBackoffMultiplier.GetAsFloat())
assert.Equal(t, "vchannelFair", params.StreamingCfg.WALBalancerPolicyName.GetValue()) assert.Equal(t, "vchannelFair", params.StreamingCfg.WALBalancerPolicyName.GetValue())
assert.Equal(t, true, params.StreamingCfg.WALBalancerPolicyAllowRebalance.GetAsBool())
assert.Equal(t, 5*time.Minute, params.StreamingCfg.WALBalancerPolicyMinRebalanceIntervalThreshold.GetAsDurationByParse())
assert.Equal(t, 1*time.Second, params.StreamingCfg.WALBalancerPolicyAllowRebalanceRecoveryLagThreshold.GetAsDurationByParse())
assert.Equal(t, 0.4, params.StreamingCfg.WALBalancerPolicyVChannelFairPChannelWeight.GetAsFloat()) assert.Equal(t, 0.4, params.StreamingCfg.WALBalancerPolicyVChannelFairPChannelWeight.GetAsFloat())
assert.Equal(t, 0.3, params.StreamingCfg.WALBalancerPolicyVChannelFairVChannelWeight.GetAsFloat()) assert.Equal(t, 0.3, params.StreamingCfg.WALBalancerPolicyVChannelFairVChannelWeight.GetAsFloat())
assert.Equal(t, 0.01, params.StreamingCfg.WALBalancerPolicyVChannelFairAntiAffinityWeight.GetAsFloat()) assert.Equal(t, 0.01, params.StreamingCfg.WALBalancerPolicyVChannelFairAntiAffinityWeight.GetAsFloat())
@ -680,6 +683,9 @@ func TestComponentParam(t *testing.T) {
params.Save(params.StreamingCfg.WALBalancerPolicyVChannelFairAntiAffinityWeight.Key, "0.02") params.Save(params.StreamingCfg.WALBalancerPolicyVChannelFairAntiAffinityWeight.Key, "0.02")
params.Save(params.StreamingCfg.WALBalancerPolicyVChannelFairRebalanceTolerance.Key, "0.02") params.Save(params.StreamingCfg.WALBalancerPolicyVChannelFairRebalanceTolerance.Key, "0.02")
params.Save(params.StreamingCfg.WALBalancerPolicyVChannelFairRebalanceMaxStep.Key, "4") params.Save(params.StreamingCfg.WALBalancerPolicyVChannelFairRebalanceMaxStep.Key, "4")
params.Save(params.StreamingCfg.WALBalancerPolicyAllowRebalance.Key, "false")
params.Save(params.StreamingCfg.WALBalancerPolicyMinRebalanceIntervalThreshold.Key, "10s")
params.Save(params.StreamingCfg.WALBalancerPolicyAllowRebalanceRecoveryLagThreshold.Key, "1s")
params.Save(params.StreamingCfg.LoggingAppendSlowThreshold.Key, "3s") params.Save(params.StreamingCfg.LoggingAppendSlowThreshold.Key, "3s")
params.Save(params.StreamingCfg.WALRecoveryGracefulCloseTimeout.Key, "4s") params.Save(params.StreamingCfg.WALRecoveryGracefulCloseTimeout.Key, "4s")
params.Save(params.StreamingCfg.WALRecoveryMaxDirtyMessage.Key, "200") params.Save(params.StreamingCfg.WALRecoveryMaxDirtyMessage.Key, "200")
@ -694,6 +700,9 @@ func TestComponentParam(t *testing.T) {
assert.Equal(t, 3.5, params.StreamingCfg.WALBalancerBackoffMultiplier.GetAsFloat()) assert.Equal(t, 3.5, params.StreamingCfg.WALBalancerBackoffMultiplier.GetAsFloat())
assert.Equal(t, 1.5, params.StreamingCfg.WALBroadcasterConcurrencyRatio.GetAsFloat()) assert.Equal(t, 1.5, params.StreamingCfg.WALBroadcasterConcurrencyRatio.GetAsFloat())
assert.Equal(t, "pchannelFair", params.StreamingCfg.WALBalancerPolicyName.GetValue()) assert.Equal(t, "pchannelFair", params.StreamingCfg.WALBalancerPolicyName.GetValue())
assert.Equal(t, false, params.StreamingCfg.WALBalancerPolicyAllowRebalance.GetAsBool())
assert.Equal(t, 10*time.Second, params.StreamingCfg.WALBalancerPolicyMinRebalanceIntervalThreshold.GetAsDurationByParse())
assert.Equal(t, 1*time.Second, params.StreamingCfg.WALBalancerPolicyAllowRebalanceRecoveryLagThreshold.GetAsDurationByParse())
assert.Equal(t, 0.5, params.StreamingCfg.WALBalancerPolicyVChannelFairPChannelWeight.GetAsFloat()) assert.Equal(t, 0.5, params.StreamingCfg.WALBalancerPolicyVChannelFairPChannelWeight.GetAsFloat())
assert.Equal(t, 0.4, params.StreamingCfg.WALBalancerPolicyVChannelFairVChannelWeight.GetAsFloat()) assert.Equal(t, 0.4, params.StreamingCfg.WALBalancerPolicyVChannelFairVChannelWeight.GetAsFloat())
assert.Equal(t, 0.02, params.StreamingCfg.WALBalancerPolicyVChannelFairAntiAffinityWeight.GetAsFloat()) assert.Equal(t, 0.02, params.StreamingCfg.WALBalancerPolicyVChannelFairAntiAffinityWeight.GetAsFloat())

View File

@ -53,6 +53,7 @@ func (s *BalanceTestSuit) SetupSuite() {
s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.BalanceCheckInterval.Key, "100") s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.BalanceCheckInterval.Key, "100")
s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.AutoBalanceInterval.Key, "100") s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.AutoBalanceInterval.Key, "100")
s.WithMilvusConfig(paramtable.Get().QueryNodeCfg.GracefulStopTimeout.Key, "1") s.WithMilvusConfig(paramtable.Get().QueryNodeCfg.GracefulStopTimeout.Key, "1")
s.WithMilvusConfig(paramtable.Get().StreamingCfg.WALBalancerPolicyMinRebalanceIntervalThreshold.Key, "1ms")
// disable compaction // disable compaction
s.WithMilvusConfig(paramtable.Get().DataCoordCfg.EnableCompaction.Key, "false") s.WithMilvusConfig(paramtable.Get().DataCoordCfg.EnableCompaction.Key, "false")

View File

@ -53,6 +53,7 @@ func (s *ChannelExclusiveBalanceSuit) SetupSuite() {
s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.Balancer.Key, meta.ChannelLevelScoreBalancerName) s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.Balancer.Key, meta.ChannelLevelScoreBalancerName)
s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.ChannelExclusiveNodeFactor.Key, "2") s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.ChannelExclusiveNodeFactor.Key, "2")
s.WithMilvusConfig(paramtable.Get().DataCoordCfg.EnableCompaction.Key, "false") s.WithMilvusConfig(paramtable.Get().DataCoordCfg.EnableCompaction.Key, "false")
s.WithMilvusConfig(paramtable.Get().StreamingCfg.WALBalancerPolicyMinRebalanceIntervalThreshold.Key, "1ms")
s.MiniClusterSuite.SetupSuite() s.MiniClusterSuite.SetupSuite()
} }

View File

@ -650,7 +650,7 @@ func (c *MiniClusterV3) getOptions() []process.Option {
env := map[string]string{ env := map[string]string{
"QUOTAANDLIMITS_ENABLED": "false", // disable the quota limits by default to avoid rpc failure. "QUOTAANDLIMITS_ENABLED": "false", // disable the quota limits by default to avoid rpc failure.
"PROXY_IP": "localhost", "PROXY_IP": "localhost",
"STRAEMINGNODE_IP": "localhost", "STREAMINGNODE_IP": "localhost",
"QUERYNODE_IP": "localhost", "QUERYNODE_IP": "localhost",
"DATANODE_IP": "localhost", "DATANODE_IP": "localhost",
"ROOTCOORD_IP": "localhost", "ROOTCOORD_IP": "localhost",

View File

@ -36,6 +36,7 @@ func (s *PartialSearchTestSuit) SetupSuite() {
s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.AutoBalanceInterval.Key, "10000") s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.AutoBalanceInterval.Key, "10000")
s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.BalanceCheckInterval.Key, "10000") s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.BalanceCheckInterval.Key, "10000")
s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.TaskExecutionCap.Key, "1") s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.TaskExecutionCap.Key, "1")
s.WithMilvusConfig(paramtable.Get().StreamingCfg.WALBalancerPolicyMinRebalanceIntervalThreshold.Key, "1ms")
s.WithOptions(integration.WithDropAllCollectionsWhenTestTearDown()) s.WithOptions(integration.WithDropAllCollectionsWhenTestTearDown())
s.MiniClusterSuite.SetupSuite() s.MiniClusterSuite.SetupSuite()

View File

@ -48,6 +48,8 @@ type ReplicaTestSuit struct {
func (s *ReplicaTestSuit) SetupSuite() { func (s *ReplicaTestSuit) SetupSuite() {
s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.BalanceCheckInterval.Key, "1000") s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.BalanceCheckInterval.Key, "1000")
s.WithMilvusConfig(paramtable.Get().QueryNodeCfg.GracefulStopTimeout.Key, "1") s.WithMilvusConfig(paramtable.Get().QueryNodeCfg.GracefulStopTimeout.Key, "1")
s.WithMilvusConfig(paramtable.Get().StreamingCfg.WALBalancerPolicyMinRebalanceIntervalThreshold.Key, "1ms")
s.MiniClusterSuite.SetupSuite() s.MiniClusterSuite.SetupSuite()
} }

View File

@ -57,6 +57,7 @@ func (s *LoadTestSuite) SetupSuite() {
s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.BalanceCheckInterval.Key, "100") s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.BalanceCheckInterval.Key, "100")
s.WithMilvusConfig(paramtable.Get().QueryNodeCfg.GracefulStopTimeout.Key, "1") s.WithMilvusConfig(paramtable.Get().QueryNodeCfg.GracefulStopTimeout.Key, "1")
s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.CheckNodeInReplicaInterval.Key, "1") s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.CheckNodeInReplicaInterval.Key, "1")
s.WithMilvusConfig(paramtable.Get().StreamingCfg.WALBalancerPolicyMinRebalanceIntervalThreshold.Key, "1ms")
s.WithOptions(integration.WithDropAllCollectionsWhenTestTearDown()) s.WithOptions(integration.WithDropAllCollectionsWhenTestTearDown())
s.WithOptions(integration.WithoutResetDeploymentWhenTestTearDown()) s.WithOptions(integration.WithoutResetDeploymentWhenTestTearDown())

View File

@ -45,6 +45,8 @@ type ManualRollingUpgradeSuite struct {
func (s *ManualRollingUpgradeSuite) SetupSuite() { func (s *ManualRollingUpgradeSuite) SetupSuite() {
rand.Seed(time.Now().UnixNano()) rand.Seed(time.Now().UnixNano())
s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.BalanceCheckInterval.Key, "100") s.WithMilvusConfig(paramtable.Get().QueryCoordCfg.BalanceCheckInterval.Key, "100")
s.WithMilvusConfig(paramtable.Get().StreamingCfg.WALBalancerPolicyMinRebalanceIntervalThreshold.Key, "1ms")
s.MiniClusterSuite.SetupSuite() s.MiniClusterSuite.SetupSuite()
} }