-
Notifications
You must be signed in to change notification settings - Fork 3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
enhance: add rw/ro streaming query node replica management #38677
base: master
Are you sure you want to change the base?
Changes from all commits
caf826c
938c00b
e92da79
6f6ff4c
da9f863
ebfa900
9b194e9
058e6dd
9a216d0
b218f49
e0026cd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,106 @@ | ||
package snmanager | ||
|
||
import ( | ||
"context" | ||
"sync" | ||
|
||
"github.com/cockroachdb/errors" | ||
"go.uber.org/zap" | ||
|
||
"github.com/milvus-io/milvus/internal/streamingcoord/server/balancer" | ||
"github.com/milvus-io/milvus/pkg/log" | ||
"github.com/milvus-io/milvus/pkg/streaming/util/types" | ||
"github.com/milvus-io/milvus/pkg/util/funcutil" | ||
"github.com/milvus-io/milvus/pkg/util/syncutil" | ||
"github.com/milvus-io/milvus/pkg/util/typeutil" | ||
) | ||
|
||
var StaticStreamingNodeManager = newStreamingNodeManager() | ||
|
||
func newStreamingNodeManager() *StreamingNodeManager { | ||
snm := &StreamingNodeManager{ | ||
notifier: syncutil.NewAsyncTaskNotifier[struct{}](), | ||
balancer: syncutil.NewFuture[balancer.Balancer](), | ||
cond: syncutil.NewContextCond(&sync.Mutex{}), | ||
latestAssignments: make(map[string]types.PChannelInfoAssigned), | ||
streamingNodes: typeutil.NewUniqueSet(), | ||
nodeChangedNotifier: syncutil.NewVersionedNotifier(), | ||
} | ||
go snm.execute() | ||
return snm | ||
} | ||
|
||
// StreamingNodeManager is a manager for manage the querynode that embedded into streaming node. | ||
// StreamingNodeManager is exclusive with ResourceManager. | ||
type StreamingNodeManager struct { | ||
notifier *syncutil.AsyncTaskNotifier[struct{}] | ||
balancer *syncutil.Future[balancer.Balancer] | ||
// The coord is merged after 2.6, so we don't need to make distribution safe. | ||
cond *syncutil.ContextCond | ||
latestAssignments map[string]types.PChannelInfoAssigned // The latest assignments info got from streaming coord balance module. | ||
streamingNodes typeutil.UniqueSet | ||
nodeChangedNotifier *syncutil.VersionedNotifier // used to notify that node in streaming node manager has been changed. | ||
} | ||
|
||
// GetWALLocated returns the server id of the node that the wal of the vChannel is located. | ||
func (s *StreamingNodeManager) GetWALLocated(vChannel string) int64 { | ||
pchannel := funcutil.ToPhysicalChannel(vChannel) | ||
var targetServerID int64 | ||
|
||
s.cond.L.Lock() | ||
for { | ||
if assignment, ok := s.latestAssignments[pchannel]; ok { | ||
targetServerID = assignment.Node.ServerID | ||
break | ||
} | ||
s.cond.Wait(context.Background()) | ||
} | ||
s.cond.L.Unlock() | ||
return targetServerID | ||
} | ||
|
||
// GetStreamingQueryNodeIDs returns the server ids of the streaming query nodes. | ||
func (s *StreamingNodeManager) GetStreamingQueryNodeIDs() typeutil.UniqueSet { | ||
s.cond.L.Lock() | ||
defer s.cond.L.Unlock() | ||
return s.streamingNodes.Clone() | ||
} | ||
|
||
// ListenNodeChanged returns a listener for node changed event. | ||
func (s *StreamingNodeManager) ListenNodeChanged() *syncutil.VersionedListener { | ||
return s.nodeChangedNotifier.Listen(syncutil.VersionedListenAtEarliest) | ||
} | ||
|
||
// SetBalancerReady set the balancer ready for the streaming node manager from streamingcoord initialization. | ||
func (s *StreamingNodeManager) SetBalancerReady(b balancer.Balancer) { | ||
s.balancer.Set(b) | ||
} | ||
|
||
func (s *StreamingNodeManager) execute() (err error) { | ||
defer s.notifier.Finish(struct{}{}) | ||
|
||
balancer, err := s.balancer.GetWithContext(s.notifier.Context()) | ||
if err != nil { | ||
return errors.Wrap(err, "failed to wait balancer ready") | ||
} | ||
for { | ||
if err := balancer.WatchChannelAssignments(s.notifier.Context(), func( | ||
version typeutil.VersionInt64Pair, | ||
relations []types.PChannelInfoAssigned, | ||
) error { | ||
s.cond.LockAndBroadcast() | ||
s.latestAssignments = make(map[string]types.PChannelInfoAssigned) | ||
s.streamingNodes = typeutil.NewUniqueSet() | ||
for _, relation := range relations { | ||
s.latestAssignments[relation.Channel.Name] = relation | ||
s.streamingNodes.Insert(relation.Node.ServerID) | ||
} | ||
s.nodeChangedNotifier.NotifyAll() | ||
log.Info("streaming node manager updated", zap.Any("assignments", s.latestAssignments), zap.Any("streamingNodes", s.streamingNodes)) | ||
s.cond.L.Unlock() | ||
return nil | ||
}); err != nil { | ||
return err | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,62 @@ | ||
package snmanager | ||
|
||
import ( | ||
"context" | ||
"testing" | ||
|
||
"github.com/stretchr/testify/assert" | ||
"github.com/stretchr/testify/mock" | ||
|
||
"github.com/milvus-io/milvus/internal/mocks/streamingcoord/server/mock_balancer" | ||
"github.com/milvus-io/milvus/pkg/streaming/util/types" | ||
"github.com/milvus-io/milvus/pkg/util/typeutil" | ||
) | ||
|
||
type pChannelInfoAssigned struct { | ||
version typeutil.VersionInt64Pair | ||
pchannels []types.PChannelInfoAssigned | ||
} | ||
|
||
func TestStreamingNodeManager(t *testing.T) { | ||
m := newStreamingNodeManager() | ||
b := mock_balancer.NewMockBalancer(t) | ||
|
||
ch := make(chan pChannelInfoAssigned, 1) | ||
b.EXPECT().WatchChannelAssignments(mock.Anything, mock.Anything).Run( | ||
func(ctx context.Context, cb func(typeutil.VersionInt64Pair, []types.PChannelInfoAssigned) error) { | ||
for { | ||
select { | ||
case <-ctx.Done(): | ||
return | ||
case p := <-ch: | ||
cb(p.version, p.pchannels) | ||
} | ||
} | ||
}) | ||
m.SetBalancerReady(b) | ||
|
||
streamingNodes := m.GetStreamingQueryNodeIDs() | ||
assert.Empty(t, streamingNodes) | ||
|
||
ch <- pChannelInfoAssigned{ | ||
version: typeutil.VersionInt64Pair{ | ||
Global: 1, | ||
Local: 1, | ||
}, | ||
pchannels: []types.PChannelInfoAssigned{ | ||
{ | ||
Channel: types.PChannelInfo{Name: "a_test", Term: 1}, | ||
Node: types.StreamingNodeInfo{ServerID: 1, Address: "localhost:1"}, | ||
}, | ||
}, | ||
} | ||
|
||
listener := m.ListenNodeChanged() | ||
err := listener.Wait(context.Background()) | ||
assert.NoError(t, err) | ||
|
||
node := m.GetWALLocated("a_test") | ||
assert.Equal(t, node, int64(1)) | ||
streamingNodes = m.GetStreamingQueryNodeIDs() | ||
assert.Equal(t, len(streamingNodes), 1) | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -27,6 +27,7 @@ | |
"github.com/milvus-io/milvus/internal/querycoordv2/meta" | ||
"github.com/milvus-io/milvus/internal/querycoordv2/session" | ||
"github.com/milvus-io/milvus/internal/querycoordv2/task" | ||
"github.com/milvus-io/milvus/internal/util/streamingutil" | ||
"github.com/milvus-io/milvus/pkg/util/paramtable" | ||
) | ||
|
||
|
@@ -105,6 +106,8 @@ | |
} | ||
|
||
func (b *RoundRobinBalancer) AssignChannel(ctx context.Context, collectionID int64, channels []*meta.DmChannel, nodes []int64, forceAssign bool) []ChannelAssignPlan { | ||
nodes = filterSQNIfStreamingServiceEnabled(nodes) | ||
|
||
// skip out suspend node and stopping node during assignment, but skip this check for manual balance | ||
if !forceAssign { | ||
versionRangeFilter := semver.MustParseRange(">2.3.x") | ||
|
@@ -119,22 +122,29 @@ | |
if len(nodesInfo) == 0 { | ||
return nil | ||
} | ||
|
||
plans := make([]ChannelAssignPlan, 0) | ||
scoreDelta := make(map[int64]int) | ||
if streamingutil.IsStreamingServiceEnabled() { | ||
channels, plans, scoreDelta = assignChannelToWALLocatedFirstForNodeInfo(channels, nodesInfo) | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. should we add a There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For one channel, Make a boost priority here is hard to achieve previous logic, because different channel has different boost for different sqn. |
||
|
||
sort.Slice(nodesInfo, func(i, j int) bool { | ||
cnt1, cnt2 := nodesInfo[i].ChannelCnt(), nodesInfo[j].ChannelCnt() | ||
id1, id2 := nodesInfo[i].ID(), nodesInfo[j].ID() | ||
delta1, delta2 := b.scheduler.GetChannelTaskDelta(id1, -1), b.scheduler.GetChannelTaskDelta(id2, -1) | ||
delta1, delta2 := b.scheduler.GetChannelTaskDelta(id1, -1)+scoreDelta[id1], b.scheduler.GetChannelTaskDelta(id2, -1)+scoreDelta[id2] | ||
return cnt1+delta1 < cnt2+delta2 | ||
}) | ||
ret := make([]ChannelAssignPlan, 0, len(channels)) | ||
|
||
for i, c := range channels { | ||
plan := ChannelAssignPlan{ | ||
Channel: c, | ||
From: -1, | ||
To: nodesInfo[i%len(nodesInfo)].ID(), | ||
} | ||
ret = append(ret, plan) | ||
plans = append(plans, plan) | ||
} | ||
return ret | ||
return plans | ||
} | ||
|
||
func (b *RoundRobinBalancer) BalanceReplica(ctx context.Context, replica *meta.Replica) ([]SegmentAssignPlan, []ChannelAssignPlan) { | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -28,6 +28,7 @@ | |
"github.com/milvus-io/milvus/internal/querycoordv2/meta" | ||
"github.com/milvus-io/milvus/internal/querycoordv2/session" | ||
"github.com/milvus-io/milvus/internal/querycoordv2/task" | ||
"github.com/milvus-io/milvus/internal/util/streamingutil" | ||
"github.com/milvus-io/milvus/pkg/log" | ||
"github.com/milvus-io/milvus/pkg/util/paramtable" | ||
"github.com/milvus-io/milvus/pkg/util/typeutil" | ||
|
@@ -67,6 +68,17 @@ | |
} | ||
}() | ||
|
||
if streamingutil.IsStreamingServiceEnabled() { | ||
// Make a plan to rebalance the channel first. | ||
// The Streaming QueryNode doesn't make the channel level score, so just fallback to the ScoreBasedBalancer. | ||
stoppingBalance := paramtable.Get().QueryCoordCfg.EnableStoppingBalance.GetAsBool() | ||
channelPlan := b.ScoreBasedBalancer.balanceChannels(ctx, br, replica, stoppingBalance) | ||
// If the channelPlan is not empty, do it directly, don't do the segment balance. | ||
if len(channelPlan) > 0 { | ||
return nil, channelPlan | ||
} | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. make sure balance channel only happens in |
||
|
||
exclusiveMode := true | ||
channels := b.targetMgr.GetDmChannelsByCollection(ctx, replica.GetCollectionID(), meta.CurrentTarget) | ||
for channelName := range channels { | ||
|
@@ -122,15 +134,15 @@ | |
zap.Any("available nodes", rwNodes), | ||
) | ||
// handle stopped nodes here, have to assign segments on stopping nodes to nodes with the smallest score | ||
if b.permitBalanceChannel(replica.GetCollectionID()) { | ||
if b.permitBalanceChannel(replica.GetCollectionID()) && !streamingutil.IsStreamingServiceEnabled() { | ||
channelPlans = append(channelPlans, b.genStoppingChannelPlan(ctx, replica, channelName, rwNodes, roNodes)...) | ||
} | ||
|
||
if len(channelPlans) == 0 && b.permitBalanceSegment(replica.GetCollectionID()) { | ||
segmentPlans = append(segmentPlans, b.genStoppingSegmentPlan(ctx, replica, channelName, rwNodes, roNodes)...) | ||
} | ||
} else { | ||
if paramtable.Get().QueryCoordCfg.AutoBalanceChannel.GetAsBool() && b.permitBalanceChannel(replica.GetCollectionID()) { | ||
if paramtable.Get().QueryCoordCfg.AutoBalanceChannel.GetAsBool() && b.permitBalanceChannel(replica.GetCollectionID()) && !streamingutil.IsStreamingServiceEnabled() { | ||
channelPlans = append(channelPlans, b.genChannelPlan(ctx, replica, channelName, rwNodes)...) | ||
} | ||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
recommand to filter the param
nodes
and print logs for invalid nodes, to make sure that channels can only be assign tosqNode
.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I will add
filterSQNIfStreamingServiceEnabled
to the entrance for everyassignChannel
operation.