mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-08 01:58:34 +08:00
Signed-off-by: MrPresent-Han <jamesharden11122@gmail.com>
This commit is contained in:
parent
bd054d90c9
commit
77c9e33e70
@ -257,6 +257,8 @@ indexNode:
|
|||||||
dataCoord:
|
dataCoord:
|
||||||
channel:
|
channel:
|
||||||
watchTimeoutInterval: 30 # Timeout on watching channels (in seconds). Datanode tickler update watch progress will reset timeout timer.
|
watchTimeoutInterval: 30 # Timeout on watching channels (in seconds). Datanode tickler update watch progress will reset timeout timer.
|
||||||
|
balanceSilentDuration: 300 # The duration before the channelBalancer on datacoord to run
|
||||||
|
balanceInterval: 360 #The interval for the channelBalancer on datacoord to check balance status
|
||||||
segment:
|
segment:
|
||||||
maxSize: 512 # Maximum size of a segment in MB
|
maxSize: 512 # Maximum size of a segment in MB
|
||||||
diskSegmentMaxSize: 2048 # Maximun size of a segment in MB for collection which has Disk index
|
diskSegmentMaxSize: 2048 # Maximun size of a segment in MB for collection which has Disk index
|
||||||
|
|||||||
@ -29,6 +29,7 @@ import (
|
|||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
|
|
||||||
clientv3 "go.etcd.io/etcd/client/v3"
|
clientv3 "go.etcd.io/etcd/client/v3"
|
||||||
|
"go.uber.org/atomic"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -39,6 +40,9 @@ type channelStateTimer struct {
|
|||||||
runningTimerStops sync.Map // channel name to timer stop channels
|
runningTimerStops sync.Map // channel name to timer stop channels
|
||||||
etcdWatcher clientv3.WatchChan
|
etcdWatcher clientv3.WatchChan
|
||||||
timeoutWatcher chan *ackEvent
|
timeoutWatcher chan *ackEvent
|
||||||
|
//Modifies afterwards must guarantee that runningTimerCount is updated synchronized with runningTimers
|
||||||
|
//in order to keep consistency
|
||||||
|
runningTimerCount atomic.Int32
|
||||||
}
|
}
|
||||||
|
|
||||||
func newChannelStateTimer(kv kv.MetaKv) *channelStateTimer {
|
func newChannelStateTimer(kv kv.MetaKv) *channelStateTimer {
|
||||||
@ -92,35 +96,40 @@ func (c *channelStateTimer) startOne(watchState datapb.ChannelWatchState, channe
|
|||||||
}
|
}
|
||||||
|
|
||||||
stop := make(chan struct{})
|
stop := make(chan struct{})
|
||||||
timer := time.NewTimer(timeout)
|
ticker := time.NewTimer(timeout)
|
||||||
c.removeTimers([]string{channelName})
|
c.removeTimers([]string{channelName})
|
||||||
c.runningTimerStops.Store(channelName, stop)
|
c.runningTimerStops.Store(channelName, stop)
|
||||||
c.runningTimers.Store(channelName, timer)
|
c.runningTimers.Store(channelName, ticker)
|
||||||
|
c.runningTimerCount.Inc()
|
||||||
go func() {
|
go func() {
|
||||||
log.Info("timer started",
|
log.Info("timer started",
|
||||||
zap.String("watch state", watchState.String()),
|
zap.String("watch state", watchState.String()),
|
||||||
zap.Int64("nodeID", nodeID),
|
zap.Int64("nodeID", nodeID),
|
||||||
zap.String("channel name", channelName),
|
zap.String("channel name", channelName),
|
||||||
zap.Duration("check interval", timeout))
|
zap.Duration("check interval", timeout))
|
||||||
defer timer.Stop()
|
defer ticker.Stop()
|
||||||
|
|
||||||
select {
|
select {
|
||||||
case <-timer.C:
|
case <-ticker.C:
|
||||||
// check tickle at path as :tickle/[prefix]/{channel_name}
|
// check tickle at path as :tickle/[prefix]/{channel_name}
|
||||||
|
c.removeTimers([]string{channelName})
|
||||||
log.Info("timeout and stop timer: wait for channel ACK timeout",
|
log.Info("timeout and stop timer: wait for channel ACK timeout",
|
||||||
zap.String("watch state", watchState.String()),
|
zap.String("watch state", watchState.String()),
|
||||||
zap.Int64("nodeID", nodeID),
|
zap.Int64("nodeID", nodeID),
|
||||||
zap.String("channel name", channelName),
|
zap.String("channel name", channelName),
|
||||||
zap.Duration("timeout interval", timeout))
|
zap.Duration("timeout interval", timeout),
|
||||||
|
zap.Int32("runningTimerCount", c.runningTimerCount.Load()))
|
||||||
ackType := getAckType(watchState)
|
ackType := getAckType(watchState)
|
||||||
c.notifyTimeoutWatcher(&ackEvent{ackType, channelName, nodeID})
|
c.notifyTimeoutWatcher(&ackEvent{ackType, channelName, nodeID})
|
||||||
|
return
|
||||||
case <-stop:
|
case <-stop:
|
||||||
log.Info("stop timer before timeout",
|
log.Info("stop timer before timeout",
|
||||||
zap.String("watch state", watchState.String()),
|
zap.String("watch state", watchState.String()),
|
||||||
zap.Int64("nodeID", nodeID),
|
zap.Int64("nodeID", nodeID),
|
||||||
zap.String("channel name", channelName),
|
zap.String("channel name", channelName),
|
||||||
zap.Duration("timeout interval", timeout))
|
zap.Duration("timeout interval", timeout),
|
||||||
|
zap.Int32("runningTimerCount", c.runningTimerCount.Load()))
|
||||||
|
return
|
||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
}
|
}
|
||||||
@ -134,6 +143,9 @@ func (c *channelStateTimer) removeTimers(channels []string) {
|
|||||||
if stop, ok := c.runningTimerStops.LoadAndDelete(channel); ok {
|
if stop, ok := c.runningTimerStops.LoadAndDelete(channel); ok {
|
||||||
close(stop.(chan struct{}))
|
close(stop.(chan struct{}))
|
||||||
c.runningTimers.Delete(channel)
|
c.runningTimers.Delete(channel)
|
||||||
|
c.runningTimerCount.Dec()
|
||||||
|
log.Info("remove timer for channel", zap.String("channel", channel),
|
||||||
|
zap.Int32("timerCount", c.runningTimerCount.Load()))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -143,6 +155,9 @@ func (c *channelStateTimer) stopIfExist(e *ackEvent) {
|
|||||||
if ok && e.ackType != watchTimeoutAck && e.ackType != releaseTimeoutAck {
|
if ok && e.ackType != watchTimeoutAck && e.ackType != releaseTimeoutAck {
|
||||||
close(stop.(chan struct{}))
|
close(stop.(chan struct{}))
|
||||||
c.runningTimers.Delete(e.channelName)
|
c.runningTimers.Delete(e.channelName)
|
||||||
|
c.runningTimerCount.Dec()
|
||||||
|
log.Info("stop timer for channel", zap.String("channel", e.channelName),
|
||||||
|
zap.Int32("timerCount", c.runningTimerCount.Load()))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -153,6 +168,12 @@ func (c *channelStateTimer) resetIfExist(channel string, interval time.Duration)
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Note here the reading towards c.running are not protected by mutex
|
||||||
|
// because it's meaningless, since we cannot guarantee the following add/delete node operations
|
||||||
|
func (c *channelStateTimer) hasRunningTimers() bool {
|
||||||
|
return c.runningTimerCount.Load() != 0
|
||||||
|
}
|
||||||
|
|
||||||
func parseWatchInfo(key string, data []byte) (*datapb.ChannelWatchInfo, error) {
|
func parseWatchInfo(key string, data []byte) (*datapb.ChannelWatchInfo, error) {
|
||||||
watchInfo := datapb.ChannelWatchInfo{}
|
watchInfo := datapb.ChannelWatchInfo{}
|
||||||
if err := proto.Unmarshal(data, &watchInfo); err != nil {
|
if err := proto.Unmarshal(data, &watchInfo); err != nil {
|
||||||
|
|||||||
@ -37,8 +37,6 @@ import (
|
|||||||
"stathat.com/c/consistent"
|
"stathat.com/c/consistent"
|
||||||
)
|
)
|
||||||
|
|
||||||
const bgCheckInterval = 3 * time.Second
|
|
||||||
|
|
||||||
// ChannelManager manages the allocation and the balance between channels and data nodes.
|
// ChannelManager manages the allocation and the balance between channels and data nodes.
|
||||||
type ChannelManager struct {
|
type ChannelManager struct {
|
||||||
ctx context.Context
|
ctx context.Context
|
||||||
@ -51,11 +49,14 @@ type ChannelManager struct {
|
|||||||
assignPolicy ChannelAssignPolicy
|
assignPolicy ChannelAssignPolicy
|
||||||
reassignPolicy ChannelReassignPolicy
|
reassignPolicy ChannelReassignPolicy
|
||||||
bgChecker ChannelBGChecker
|
bgChecker ChannelBGChecker
|
||||||
|
balancePolicy BalanceChannelPolicy
|
||||||
msgstreamFactory msgstream.Factory
|
msgstreamFactory msgstream.Factory
|
||||||
|
|
||||||
stateChecker channelStateChecker
|
stateChecker channelStateChecker
|
||||||
stopChecker context.CancelFunc
|
stopChecker context.CancelFunc
|
||||||
stateTimer *channelStateTimer
|
stateTimer *channelStateTimer
|
||||||
|
|
||||||
|
lastActiveTimestamp time.Time
|
||||||
}
|
}
|
||||||
|
|
||||||
type channel struct {
|
type channel struct {
|
||||||
@ -90,6 +91,10 @@ func withStateChecker() ChannelManagerOpt {
|
|||||||
return func(c *ChannelManager) { c.stateChecker = c.watchChannelStatesLoop }
|
return func(c *ChannelManager) { c.stateChecker = c.watchChannelStatesLoop }
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func withBgChecker() ChannelManagerOpt {
|
||||||
|
return func(c *ChannelManager) { c.bgChecker = c.bgCheckChannelsWork }
|
||||||
|
}
|
||||||
|
|
||||||
// NewChannelManager creates and returns a new ChannelManager instance.
|
// NewChannelManager creates and returns a new ChannelManager instance.
|
||||||
func NewChannelManager(
|
func NewChannelManager(
|
||||||
kv kv.MetaKv, // for TxnKv and MetaKv
|
kv kv.MetaKv, // for TxnKv and MetaKv
|
||||||
@ -116,7 +121,8 @@ func NewChannelManager(
|
|||||||
c.deregisterPolicy = c.factory.NewDeregisterPolicy()
|
c.deregisterPolicy = c.factory.NewDeregisterPolicy()
|
||||||
c.assignPolicy = c.factory.NewAssignPolicy()
|
c.assignPolicy = c.factory.NewAssignPolicy()
|
||||||
c.reassignPolicy = c.factory.NewReassignPolicy()
|
c.reassignPolicy = c.factory.NewReassignPolicy()
|
||||||
c.bgChecker = c.factory.NewBgChecker()
|
c.balancePolicy = c.factory.NewBalancePolicy()
|
||||||
|
c.lastActiveTimestamp = time.Now()
|
||||||
return c, nil
|
return c, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -155,13 +161,18 @@ func (c *ChannelManager) Startup(ctx context.Context, nodes []int64) error {
|
|||||||
// Unwatch and drop channel with drop flag.
|
// Unwatch and drop channel with drop flag.
|
||||||
c.unwatchDroppedChannels()
|
c.unwatchDroppedChannels()
|
||||||
|
|
||||||
|
checkerContext, cancel := context.WithCancel(ctx)
|
||||||
|
c.stopChecker = cancel
|
||||||
if c.stateChecker != nil {
|
if c.stateChecker != nil {
|
||||||
ctx1, cancel := context.WithCancel(ctx)
|
go c.stateChecker(checkerContext)
|
||||||
c.stopChecker = cancel
|
|
||||||
go c.stateChecker(ctx1)
|
|
||||||
log.Info("starting etcd states checker")
|
log.Info("starting etcd states checker")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if c.bgChecker != nil {
|
||||||
|
go c.bgChecker(checkerContext)
|
||||||
|
log.Info("starting background balance checker")
|
||||||
|
}
|
||||||
|
|
||||||
log.Info("cluster start up",
|
log.Info("cluster start up",
|
||||||
zap.Int64s("nodes", nodes),
|
zap.Int64s("nodes", nodes),
|
||||||
zap.Int64s("oNodes", oNodes),
|
zap.Int64s("oNodes", oNodes),
|
||||||
@ -246,38 +257,25 @@ func (c *ChannelManager) unwatchDroppedChannels() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// NOT USED.
|
|
||||||
func (c *ChannelManager) bgCheckChannelsWork(ctx context.Context) {
|
func (c *ChannelManager) bgCheckChannelsWork(ctx context.Context) {
|
||||||
ticker := time.NewTicker(bgCheckInterval)
|
ticker := time.NewTicker(Params.DataCoordCfg.ChannelBalanceInterval.GetAsDuration(time.Second))
|
||||||
defer ticker.Stop()
|
defer ticker.Stop()
|
||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case <-ctx.Done():
|
case <-ctx.Done():
|
||||||
|
log.Info("background checking channels loop quit")
|
||||||
return
|
return
|
||||||
case <-ticker.C:
|
case <-ticker.C:
|
||||||
c.mu.Lock()
|
c.mu.Lock()
|
||||||
|
if !c.isSilent() {
|
||||||
channels := c.store.GetNodesChannels()
|
log.Info("ChannelManager is not silent, skip channel balance this round")
|
||||||
reallocates, err := c.bgChecker(channels, time.Now())
|
} else {
|
||||||
if err != nil {
|
toReleases := c.balancePolicy(c.store, time.Now())
|
||||||
log.Warn("channel manager bg check failed", zap.Error(err))
|
log.Info("channel manager bg check balance", zap.Array("toReleases", toReleases))
|
||||||
|
if err := c.updateWithTimer(toReleases, datapb.ChannelWatchState_ToRelease); err != nil {
|
||||||
c.mu.Unlock()
|
log.Warn("channel store update error", zap.Error(err))
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
updates := c.reassignPolicy(c.store, reallocates)
|
|
||||||
log.Info("channel manager bg check reassign", zap.Array("updates", updates))
|
|
||||||
for _, update := range updates {
|
|
||||||
if update.Type == Add {
|
|
||||||
c.fillChannelWatchInfo(update)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if err := c.store.Update(updates); err != nil {
|
|
||||||
log.Warn("channel store update error", zap.Error(err))
|
|
||||||
}
|
|
||||||
|
|
||||||
c.mu.Unlock()
|
c.mu.Unlock()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -610,6 +608,7 @@ func (c *ChannelManager) updateWithTimer(updates ChannelOpSet, state datapb.Chan
|
|||||||
log.Warn("fail to update", zap.Array("updates", updates), zap.Error(err))
|
log.Warn("fail to update", zap.Array("updates", updates), zap.Error(err))
|
||||||
c.stateTimer.removeTimers(channelsWithTimer)
|
c.stateTimer.removeTimers(channelsWithTimer)
|
||||||
}
|
}
|
||||||
|
c.lastActiveTimestamp = time.Now()
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -623,14 +622,17 @@ func (c *ChannelManager) processAck(e *ackEvent) {
|
|||||||
case watchSuccessAck:
|
case watchSuccessAck:
|
||||||
log.Info("datanode successfully watched channel", zap.Int64("nodeID", e.nodeID), zap.String("channel name", e.channelName))
|
log.Info("datanode successfully watched channel", zap.Int64("nodeID", e.nodeID), zap.String("channel name", e.channelName))
|
||||||
case watchFailAck, watchTimeoutAck: // failure acks from toWatch
|
case watchFailAck, watchTimeoutAck: // failure acks from toWatch
|
||||||
|
log.Warn("datanode watch channel failed or timeout, will release", zap.Int64("nodeID", e.nodeID),
|
||||||
|
zap.String("channel", e.channelName))
|
||||||
err := c.Release(e.nodeID, e.channelName)
|
err := c.Release(e.nodeID, e.channelName)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("fail to set channels to release for watch failure ACKs",
|
log.Warn("fail to set channels to release for watch failure ACKs",
|
||||||
zap.Int64("nodeID", e.nodeID), zap.String("channel name", e.channelName), zap.Error(err))
|
zap.Int64("nodeID", e.nodeID), zap.String("channel name", e.channelName), zap.Error(err))
|
||||||
}
|
}
|
||||||
|
|
||||||
case releaseFailAck, releaseTimeoutAck: // failure acks from toRelease
|
case releaseFailAck, releaseTimeoutAck: // failure acks from toRelease
|
||||||
// Cleanup, Delete and Reassign
|
// Cleanup, Delete and Reassign
|
||||||
|
log.Warn("datanode release channel failed or timeout, will cleanup and reassign", zap.Int64("nodeID", e.nodeID),
|
||||||
|
zap.String("channel", e.channelName))
|
||||||
err := c.CleanupAndReassign(e.nodeID, e.channelName)
|
err := c.CleanupAndReassign(e.nodeID, e.channelName)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("fail to clean and reassign channels for release failure ACKs",
|
log.Warn("fail to clean and reassign channels for release failure ACKs",
|
||||||
@ -639,6 +641,8 @@ func (c *ChannelManager) processAck(e *ackEvent) {
|
|||||||
|
|
||||||
case releaseSuccessAck:
|
case releaseSuccessAck:
|
||||||
// Delete and Reassign
|
// Delete and Reassign
|
||||||
|
log.Info("datanode release channel successfully, will reassign", zap.Int64("nodeID", e.nodeID),
|
||||||
|
zap.String("channel", e.channelName))
|
||||||
err := c.Reassign(e.nodeID, e.channelName)
|
err := c.Reassign(e.nodeID, e.channelName)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("fail to response to release success ACK",
|
log.Warn("fail to response to release success ACK",
|
||||||
@ -739,19 +743,19 @@ func (c *ChannelManager) Release(nodeID UniqueID, channelName string) error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Reassign reassigns a channel to another DataNode.
|
// Reassign reassigns a channel to another DataNode.
|
||||||
func (c *ChannelManager) Reassign(nodeID UniqueID, channelName string) error {
|
func (c *ChannelManager) Reassign(originNodeID UniqueID, channelName string) error {
|
||||||
c.mu.Lock()
|
c.mu.Lock()
|
||||||
defer c.mu.Unlock()
|
defer c.mu.Unlock()
|
||||||
|
|
||||||
ch := c.getChannelByNodeAndName(nodeID, channelName)
|
ch := c.getChannelByNodeAndName(originNodeID, channelName)
|
||||||
if ch == nil {
|
if ch == nil {
|
||||||
return fmt.Errorf("fail to find matching nodeID: %d with channelName: %s", nodeID, channelName)
|
return fmt.Errorf("fail to find matching nodeID: %d with channelName: %s", originNodeID, channelName)
|
||||||
}
|
}
|
||||||
|
|
||||||
reallocates := &NodeChannelInfo{nodeID, []*channel{ch}}
|
reallocates := &NodeChannelInfo{originNodeID, []*channel{ch}}
|
||||||
|
|
||||||
if c.isMarkedDrop(channelName) {
|
if c.isMarkedDrop(channelName) {
|
||||||
if err := c.remove(nodeID, ch); err != nil {
|
if err := c.remove(originNodeID, ch); err != nil {
|
||||||
return fmt.Errorf("failed to remove watch info: %v,%s", ch, err.Error())
|
return fmt.Errorf("failed to remove watch info: %v,%s", ch, err.Error())
|
||||||
}
|
}
|
||||||
if err := c.h.FinishDropChannel(channelName); err != nil {
|
if err := c.h.FinishDropChannel(channelName); err != nil {
|
||||||
@ -766,13 +770,13 @@ func (c *ChannelManager) Reassign(nodeID UniqueID, channelName string) error {
|
|||||||
if len(updates) <= 0 {
|
if len(updates) <= 0 {
|
||||||
// Skip the remove if reassign to the original node.
|
// Skip the remove if reassign to the original node.
|
||||||
log.Warn("failed to reassign channel to other nodes, assigning to the original DataNode",
|
log.Warn("failed to reassign channel to other nodes, assigning to the original DataNode",
|
||||||
zap.Int64("nodeID", nodeID),
|
zap.Int64("nodeID", originNodeID),
|
||||||
zap.String("channel name", channelName))
|
zap.String("channel name", channelName))
|
||||||
updates.Add(nodeID, []*channel{ch})
|
updates.Add(originNodeID, []*channel{ch})
|
||||||
}
|
}
|
||||||
|
|
||||||
log.Info("channel manager reassigning channels",
|
log.Info("channel manager reassigning channels",
|
||||||
zap.Int64("old node ID", nodeID),
|
zap.Int64("old node ID", originNodeID),
|
||||||
zap.Array("updates", updates))
|
zap.Array("updates", updates))
|
||||||
return c.updateWithTimer(updates, datapb.ChannelWatchState_ToWatch)
|
return c.updateWithTimer(updates, datapb.ChannelWatchState_ToWatch)
|
||||||
}
|
}
|
||||||
@ -864,3 +868,10 @@ func getReleaseOp(nodeID UniqueID, ch *channel) ChannelOpSet {
|
|||||||
op.Add(nodeID, []*channel{ch})
|
op.Add(nodeID, []*channel{ch})
|
||||||
return op
|
return op
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (c *ChannelManager) isSilent() bool {
|
||||||
|
if c.stateTimer.hasRunningTimers() {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
return time.Since(c.lastActiveTimestamp) >= Params.DataCoordCfg.ChannelBalanceSilentDuration.GetAsDuration(time.Second)
|
||||||
|
}
|
||||||
|
|||||||
@ -31,8 +31,8 @@ type ChannelPolicyFactory interface {
|
|||||||
NewAssignPolicy() ChannelAssignPolicy
|
NewAssignPolicy() ChannelAssignPolicy
|
||||||
// NewReassignPolicy creates a new channel reassign policy.
|
// NewReassignPolicy creates a new channel reassign policy.
|
||||||
NewReassignPolicy() ChannelReassignPolicy
|
NewReassignPolicy() ChannelReassignPolicy
|
||||||
// NewBgChecker creates a new background checker.
|
// NewBalancePolicy creates a new channel balance policy.
|
||||||
NewBgChecker() ChannelBGChecker
|
NewBalancePolicy() BalanceChannelPolicy
|
||||||
}
|
}
|
||||||
|
|
||||||
// ChannelPolicyFactoryV1 equal to policy batch
|
// ChannelPolicyFactoryV1 equal to policy batch
|
||||||
@ -65,9 +65,8 @@ func (f *ChannelPolicyFactoryV1) NewReassignPolicy() ChannelReassignPolicy {
|
|||||||
return AverageReassignPolicy
|
return AverageReassignPolicy
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewBgChecker implementing ChannelPolicyFactory
|
func (f *ChannelPolicyFactoryV1) NewBalancePolicy() BalanceChannelPolicy {
|
||||||
func (f *ChannelPolicyFactoryV1) NewBgChecker() ChannelBGChecker {
|
return AvgBalanceChannelPolicy
|
||||||
return BgCheckWithMaxWatchDuration(f.kv)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// ConsistentHashChannelPolicyFactory use consistent hash to determine channel assignment
|
// ConsistentHashChannelPolicyFactory use consistent hash to determine channel assignment
|
||||||
@ -102,7 +101,7 @@ func (f *ConsistentHashChannelPolicyFactory) NewReassignPolicy() ChannelReassign
|
|||||||
return EmptyReassignPolicy
|
return EmptyReassignPolicy
|
||||||
}
|
}
|
||||||
|
|
||||||
// NewBgChecker creates a new background checker
|
// NewBalancePolicy creates a new balance policy
|
||||||
func (f *ConsistentHashChannelPolicyFactory) NewBgChecker() ChannelBGChecker {
|
func (f *ConsistentHashChannelPolicyFactory) NewBalancePolicy() BalanceChannelPolicy {
|
||||||
return EmptyBgChecker
|
return EmptyBalancePolicy
|
||||||
}
|
}
|
||||||
|
|||||||
@ -733,6 +733,45 @@ func TestChannelManager(t *testing.T) {
|
|||||||
|
|
||||||
waitAndCheckState(t, metakv, datapb.ChannelWatchState_ToWatch, nodeID, channelName, collectionID)
|
waitAndCheckState(t, metakv, datapb.ChannelWatchState_ToWatch, nodeID, channelName, collectionID)
|
||||||
})
|
})
|
||||||
|
|
||||||
|
t.Run("test background check silent", func(t *testing.T) {
|
||||||
|
metakv.RemoveWithPrefix("")
|
||||||
|
defer metakv.RemoveWithPrefix("")
|
||||||
|
prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue()
|
||||||
|
var (
|
||||||
|
collectionID = UniqueID(9)
|
||||||
|
channelNamePrefix = t.Name()
|
||||||
|
nodeID = UniqueID(111)
|
||||||
|
)
|
||||||
|
cName := channelNamePrefix + "TestBgChecker"
|
||||||
|
|
||||||
|
//1. set up channel_manager
|
||||||
|
ctx, cancel := context.WithCancel(context.TODO())
|
||||||
|
defer cancel()
|
||||||
|
chManager, err := NewChannelManager(metakv, newMockHandler(), withBgChecker())
|
||||||
|
require.NoError(t, err)
|
||||||
|
assert.NotNil(t, chManager.bgChecker)
|
||||||
|
chManager.Startup(ctx, []int64{nodeID})
|
||||||
|
|
||||||
|
//2. test isSilent function running correctly
|
||||||
|
Params.Save(Params.DataCoordCfg.ChannelBalanceSilentDuration.Key, "3")
|
||||||
|
assert.False(t, chManager.isSilent())
|
||||||
|
assert.False(t, chManager.stateTimer.hasRunningTimers())
|
||||||
|
|
||||||
|
//3. watch one channel
|
||||||
|
chManager.Watch(&channel{Name: cName, CollectionID: collectionID})
|
||||||
|
assert.False(t, chManager.isSilent())
|
||||||
|
assert.True(t, chManager.stateTimer.hasRunningTimers())
|
||||||
|
key := path.Join(prefix, strconv.FormatInt(nodeID, 10), cName)
|
||||||
|
waitAndStore(t, metakv, key, datapb.ChannelWatchState_ToWatch, datapb.ChannelWatchState_WatchSuccess)
|
||||||
|
waitAndCheckState(t, metakv, datapb.ChannelWatchState_WatchSuccess, nodeID, cName, collectionID)
|
||||||
|
|
||||||
|
//4. wait for duration and check silent again
|
||||||
|
time.Sleep(Params.DataCoordCfg.ChannelBalanceSilentDuration.GetAsDuration(time.Second))
|
||||||
|
chManager.stateTimer.removeTimers([]string{cName})
|
||||||
|
assert.True(t, chManager.isSilent())
|
||||||
|
assert.False(t, chManager.stateTimer.hasRunningTimers())
|
||||||
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestChannelManager_Reload(t *testing.T) {
|
func TestChannelManager_Reload(t *testing.T) {
|
||||||
|
|||||||
@ -28,7 +28,10 @@ import (
|
|||||||
|
|
||||||
"github.com/golang/protobuf/proto"
|
"github.com/golang/protobuf/proto"
|
||||||
"github.com/milvus-io/milvus/internal/kv"
|
"github.com/milvus-io/milvus/internal/kv"
|
||||||
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||||
|
|
||||||
|
"go.uber.org/zap"
|
||||||
"go.uber.org/zap/zapcore"
|
"go.uber.org/zap/zapcore"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -89,6 +92,8 @@ type ROChannelStore interface {
|
|||||||
GetBufferChannelInfo() *NodeChannelInfo
|
GetBufferChannelInfo() *NodeChannelInfo
|
||||||
// GetNodes gets all node ids in store.
|
// GetNodes gets all node ids in store.
|
||||||
GetNodes() []int64
|
GetNodes() []int64
|
||||||
|
// GetNodeChannelCount
|
||||||
|
GetNodeChannelCount(nodeID int64) int
|
||||||
}
|
}
|
||||||
|
|
||||||
// RWChannelStore is the read write channel store for channels and nodes.
|
// RWChannelStore is the read write channel store for channels and nodes.
|
||||||
@ -160,6 +165,8 @@ func (c *ChannelStore) Reload() error {
|
|||||||
Schema: cw.GetSchema(),
|
Schema: cw.GetSchema(),
|
||||||
}
|
}
|
||||||
c.channelsInfo[nodeID].Channels = append(c.channelsInfo[nodeID].Channels, channel)
|
c.channelsInfo[nodeID].Channels = append(c.channelsInfo[nodeID].Channels, channel)
|
||||||
|
log.Info("channel store reload channel",
|
||||||
|
zap.Int64("nodeID", nodeID), zap.String("channel", channel.Name))
|
||||||
}
|
}
|
||||||
record.Record("ChannelStore reload")
|
record.Record("ChannelStore reload")
|
||||||
return nil
|
return nil
|
||||||
@ -313,6 +320,15 @@ func (c *ChannelStore) GetNode(nodeID int64) *NodeChannelInfo {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (c *ChannelStore) GetNodeChannelCount(nodeID int64) int {
|
||||||
|
for id, info := range c.channelsInfo {
|
||||||
|
if id == nodeID {
|
||||||
|
return len(info.Channels)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return 0
|
||||||
|
}
|
||||||
|
|
||||||
// Delete removes the given node from the channel store and returns its channels.
|
// Delete removes the given node from the channel store and returns its channels.
|
||||||
func (c *ChannelStore) Delete(nodeID int64) ([]*channel, error) {
|
func (c *ChannelStore) Delete(nodeID int64) ([]*channel, error) {
|
||||||
for id, info := range c.channelsInfo {
|
for id, info := range c.channelsInfo {
|
||||||
|
|||||||
@ -17,15 +17,15 @@
|
|||||||
package datacoord
|
package datacoord
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"context"
|
||||||
|
"math"
|
||||||
"sort"
|
"sort"
|
||||||
"strconv"
|
"strconv"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/golang/protobuf/proto"
|
|
||||||
"github.com/milvus-io/milvus/internal/kv"
|
|
||||||
"github.com/milvus-io/milvus/internal/log"
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
"go.uber.org/zap/zapcore"
|
||||||
"stathat.com/c/consistent"
|
"stathat.com/c/consistent"
|
||||||
)
|
)
|
||||||
|
|
||||||
@ -364,6 +364,27 @@ func ConsistentHashDeregisterPolicy(hashRing *consistent.Consistent) DeregisterP
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
type BalanceChannelPolicy func(store ROChannelStore, ts time.Time) ChannelOpSet
|
||||||
|
|
||||||
|
func AvgBalanceChannelPolicy(store ROChannelStore, ts time.Time) ChannelOpSet {
|
||||||
|
channelOps := make(ChannelOpSet, 0)
|
||||||
|
reAllocates, err := BgBalanceCheck(store.GetNodesChannels(), ts)
|
||||||
|
if err != nil {
|
||||||
|
log.Error("failed to balance node channels", zap.Error(err))
|
||||||
|
return channelOps
|
||||||
|
}
|
||||||
|
for _, reAlloc := range reAllocates {
|
||||||
|
toRelease := &ChannelOp{
|
||||||
|
Type: Add,
|
||||||
|
Channels: reAlloc.Channels,
|
||||||
|
NodeID: reAlloc.NodeID,
|
||||||
|
}
|
||||||
|
channelOps = append(channelOps, toRelease)
|
||||||
|
}
|
||||||
|
|
||||||
|
return channelOps
|
||||||
|
}
|
||||||
|
|
||||||
// ChannelReassignPolicy is a policy for reassigning channels
|
// ChannelReassignPolicy is a policy for reassigning channels
|
||||||
type ChannelReassignPolicy func(store ROChannelStore, reassigns []*NodeChannelInfo) ChannelOpSet
|
type ChannelReassignPolicy func(store ROChannelStore, reassigns []*NodeChannelInfo) ChannelOpSet
|
||||||
|
|
||||||
@ -372,8 +393,13 @@ func EmptyReassignPolicy(store ROChannelStore, reassigns []*NodeChannelInfo) Cha
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// AverageReassignPolicy is a reassigning policy that evenly assign channels
|
// EmptyBalancePolicy is a dummy balance policy
|
||||||
func AverageReassignPolicy(store ROChannelStore, reassigns []*NodeChannelInfo) ChannelOpSet {
|
func EmptyBalancePolicy(store ROChannelStore, ts time.Time) ChannelOpSet {
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
// RoundRobinReassignPolicy is a reassigning policy that evenly assign channels
|
||||||
|
func RoundRobinReassignPolicy(store ROChannelStore, reassigns []*NodeChannelInfo) ChannelOpSet {
|
||||||
allNodes := store.GetNodesChannels()
|
allNodes := store.GetNodesChannels()
|
||||||
filterMap := make(map[int64]struct{})
|
filterMap := make(map[int64]struct{})
|
||||||
for _, reassign := range reassigns {
|
for _, reassign := range reassigns {
|
||||||
@ -386,10 +412,10 @@ func AverageReassignPolicy(store ROChannelStore, reassigns []*NodeChannelInfo) C
|
|||||||
}
|
}
|
||||||
avaNodes = append(avaNodes, c)
|
avaNodes = append(avaNodes, c)
|
||||||
}
|
}
|
||||||
|
ret := make([]*ChannelOp, 0)
|
||||||
if len(avaNodes) == 0 {
|
if len(avaNodes) == 0 {
|
||||||
// if no node is left, do not reassign
|
// if no node is left, do not reassign
|
||||||
return nil
|
return ret
|
||||||
}
|
}
|
||||||
sort.Slice(avaNodes, func(i, j int) bool {
|
sort.Slice(avaNodes, func(i, j int) bool {
|
||||||
return len(avaNodes[i].Channels) <= len(avaNodes[j].Channels)
|
return len(avaNodes[i].Channels) <= len(avaNodes[j].Channels)
|
||||||
@ -397,7 +423,6 @@ func AverageReassignPolicy(store ROChannelStore, reassigns []*NodeChannelInfo) C
|
|||||||
|
|
||||||
// reassign channels to remaining nodes
|
// reassign channels to remaining nodes
|
||||||
i := 0
|
i := 0
|
||||||
ret := make([]*ChannelOp, 0)
|
|
||||||
addUpdates := make(map[int64]*ChannelOp)
|
addUpdates := make(map[int64]*ChannelOp)
|
||||||
for _, reassign := range reassigns {
|
for _, reassign := range reassigns {
|
||||||
deleteUpdate := &ChannelOp{
|
deleteUpdate := &ChannelOp{
|
||||||
@ -427,51 +452,158 @@ func AverageReassignPolicy(store ROChannelStore, reassigns []*NodeChannelInfo) C
|
|||||||
return ret
|
return ret
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// AverageReassignPolicy is a reassigning policy that evenly balance channels among datanodes
|
||||||
|
// which is used by bgChecker
|
||||||
|
func AverageReassignPolicy(store ROChannelStore, reassigns []*NodeChannelInfo) ChannelOpSet {
|
||||||
|
allNodes := store.GetNodesChannels()
|
||||||
|
filterMap := make(map[int64]struct{})
|
||||||
|
toReassignTotalNum := 0
|
||||||
|
for _, reassign := range reassigns {
|
||||||
|
filterMap[reassign.NodeID] = struct{}{}
|
||||||
|
toReassignTotalNum += len(reassign.Channels)
|
||||||
|
}
|
||||||
|
avaNodes := make([]*NodeChannelInfo, 0, len(allNodes))
|
||||||
|
avaNodesChannelSum := 0
|
||||||
|
for _, node := range allNodes {
|
||||||
|
if _, ok := filterMap[node.NodeID]; ok {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
avaNodes = append(avaNodes, node)
|
||||||
|
avaNodesChannelSum += len(node.Channels)
|
||||||
|
}
|
||||||
|
log.Info("AverageReassignPolicy working", zap.Int("avaNodesCount", len(avaNodes)),
|
||||||
|
zap.Int("toAssignChannelNum", toReassignTotalNum), zap.Int("avaNodesChannelSum", avaNodesChannelSum))
|
||||||
|
ret := make([]*ChannelOp, 0)
|
||||||
|
if len(avaNodes) == 0 {
|
||||||
|
// if no node is left, do not reassign
|
||||||
|
log.Warn("there is no available nodes when reassigning, return")
|
||||||
|
return ret
|
||||||
|
}
|
||||||
|
|
||||||
|
avgChannelCount := int(math.Ceil(float64(avaNodesChannelSum+toReassignTotalNum) / (float64(len(avaNodes)))))
|
||||||
|
sort.Slice(avaNodes, func(i, j int) bool {
|
||||||
|
if len(avaNodes[i].Channels) == len(avaNodes[j].Channels) {
|
||||||
|
return avaNodes[i].NodeID < avaNodes[j].NodeID
|
||||||
|
}
|
||||||
|
return len(avaNodes[i].Channels) < len(avaNodes[j].Channels)
|
||||||
|
})
|
||||||
|
|
||||||
|
// reassign channels to remaining nodes
|
||||||
|
addUpdates := make(map[int64]*ChannelOp)
|
||||||
|
for _, reassign := range reassigns {
|
||||||
|
deleteUpdate := &ChannelOp{
|
||||||
|
Type: Delete,
|
||||||
|
Channels: reassign.Channels,
|
||||||
|
NodeID: reassign.NodeID,
|
||||||
|
}
|
||||||
|
ret = append(ret, deleteUpdate)
|
||||||
|
for _, ch := range reassign.Channels {
|
||||||
|
nodeIdx := 0
|
||||||
|
for {
|
||||||
|
targetID := avaNodes[nodeIdx%len(avaNodes)].NodeID
|
||||||
|
if nodeIdx < len(avaNodes) {
|
||||||
|
existedChannelCount := store.GetNodeChannelCount(targetID)
|
||||||
|
if _, ok := addUpdates[targetID]; !ok {
|
||||||
|
if existedChannelCount >= avgChannelCount {
|
||||||
|
log.Debug("targetNodeID has had more channels than average, skip", zap.Int64("targetID",
|
||||||
|
targetID), zap.Int("existedChannelCount", existedChannelCount))
|
||||||
|
nodeIdx++
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
addingChannelCount := len(addUpdates[targetID].Channels)
|
||||||
|
if existedChannelCount+addingChannelCount >= avgChannelCount {
|
||||||
|
log.Debug("targetNodeID has had more channels than average, skip", zap.Int64("targetID",
|
||||||
|
targetID), zap.Int("currentChannelCount", existedChannelCount+addingChannelCount))
|
||||||
|
nodeIdx++
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
nodeIdx++
|
||||||
|
}
|
||||||
|
if _, ok := addUpdates[targetID]; !ok {
|
||||||
|
addUpdates[targetID] = &ChannelOp{
|
||||||
|
Type: Add,
|
||||||
|
NodeID: targetID,
|
||||||
|
Channels: []*channel{ch},
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
addUpdates[targetID].Channels = append(addUpdates[targetID].Channels, ch)
|
||||||
|
}
|
||||||
|
break
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
for _, update := range addUpdates {
|
||||||
|
ret = append(ret, update)
|
||||||
|
}
|
||||||
|
return ret
|
||||||
|
}
|
||||||
|
|
||||||
// ChannelBGChecker check nodes' channels and return the channels needed to be reallocated.
|
// ChannelBGChecker check nodes' channels and return the channels needed to be reallocated.
|
||||||
type ChannelBGChecker func(channels []*NodeChannelInfo, ts time.Time) ([]*NodeChannelInfo, error)
|
type ChannelBGChecker func(ctx context.Context)
|
||||||
|
|
||||||
// EmptyBgChecker does nothing
|
// EmptyBgChecker does nothing
|
||||||
func EmptyBgChecker(channels []*NodeChannelInfo, ts time.Time) ([]*NodeChannelInfo, error) {
|
func EmptyBgChecker(channels []*NodeChannelInfo, ts time.Time) ([]*NodeChannelInfo, error) {
|
||||||
return nil, nil
|
return nil, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// BgCheckWithMaxWatchDuration returns a ChannelBGChecker with `Params.DataCoordCfg.MaxWatchDuration`.
|
type ReAllocates []*NodeChannelInfo
|
||||||
func BgCheckWithMaxWatchDuration(kv kv.TxnKV) ChannelBGChecker {
|
|
||||||
return func(channels []*NodeChannelInfo, ts time.Time) ([]*NodeChannelInfo, error) {
|
func (rallocates ReAllocates) MarshalLogArray(enc zapcore.ArrayEncoder) error {
|
||||||
reAllocations := make([]*NodeChannelInfo, 0, len(channels))
|
for _, nChannelInfo := range rallocates {
|
||||||
for _, ch := range channels {
|
enc.AppendString("nodeID:")
|
||||||
cinfo := &NodeChannelInfo{
|
enc.AppendInt64(nChannelInfo.NodeID)
|
||||||
NodeID: ch.NodeID,
|
cstr := "["
|
||||||
Channels: make([]*channel, 0),
|
if len(nChannelInfo.Channels) > 0 {
|
||||||
}
|
for _, s := range nChannelInfo.Channels {
|
||||||
for _, c := range ch.Channels {
|
cstr += s.Name
|
||||||
k := buildNodeChannelKey(ch.NodeID, c.Name)
|
cstr += ", "
|
||||||
v, err := kv.Load(k)
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
watchInfo := &datapb.ChannelWatchInfo{}
|
|
||||||
if err := proto.Unmarshal([]byte(v), watchInfo); err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
reviseVChannelInfo(watchInfo.GetVchan())
|
|
||||||
// if a channel is not watched or update watch progress after WatchTimeoutInterval,
|
|
||||||
// then we reallocate it to another node
|
|
||||||
if watchInfo.State == datapb.ChannelWatchState_Complete || watchInfo.State == datapb.ChannelWatchState_WatchSuccess {
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
startTime := time.Unix(watchInfo.StartTs, 0)
|
|
||||||
d := ts.Sub(startTime)
|
|
||||||
if d >= Params.DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second) {
|
|
||||||
cinfo.Channels = append(cinfo.Channels, c)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
if len(cinfo.Channels) != 0 {
|
|
||||||
reAllocations = append(reAllocations, cinfo)
|
|
||||||
}
|
}
|
||||||
|
cstr = cstr[:len(cstr)-2]
|
||||||
}
|
}
|
||||||
|
cstr += "]"
|
||||||
|
enc.AppendString(cstr)
|
||||||
|
}
|
||||||
|
return nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func BgBalanceCheck(nodeChannels []*NodeChannelInfo, ts time.Time) ([]*NodeChannelInfo, error) {
|
||||||
|
avaNodeNum := len(nodeChannels)
|
||||||
|
reAllocations := make(ReAllocates, 0, avaNodeNum)
|
||||||
|
if avaNodeNum == 0 {
|
||||||
return reAllocations, nil
|
return reAllocations, nil
|
||||||
}
|
}
|
||||||
|
totalChannelNum := 0
|
||||||
|
for _, nodeChs := range nodeChannels {
|
||||||
|
totalChannelNum += len(nodeChs.Channels)
|
||||||
|
}
|
||||||
|
channelCountPerNode := totalChannelNum / avaNodeNum
|
||||||
|
for _, nChannels := range nodeChannels {
|
||||||
|
chCount := len(nChannels.Channels)
|
||||||
|
if chCount <= channelCountPerNode+1 {
|
||||||
|
log.Info("node channel count is not much larger than average, skip reallocate",
|
||||||
|
zap.Int64("nodeID", nChannels.NodeID), zap.Int("channelCount", chCount),
|
||||||
|
zap.Int("channelCountPerNode", channelCountPerNode))
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
reallocate := &NodeChannelInfo{
|
||||||
|
NodeID: nChannels.NodeID,
|
||||||
|
Channels: make([]*channel, 0),
|
||||||
|
}
|
||||||
|
toReleaseCount := chCount - channelCountPerNode - 1
|
||||||
|
for _, ch := range nChannels.Channels {
|
||||||
|
reallocate.Channels = append(reallocate.Channels, ch)
|
||||||
|
toReleaseCount--
|
||||||
|
if toReleaseCount <= 0 {
|
||||||
|
break
|
||||||
|
}
|
||||||
|
}
|
||||||
|
reAllocations = append(reAllocations, reallocate)
|
||||||
|
}
|
||||||
|
log.Info("Channel Balancer got new reAllocations:", zap.Array("reAllocations", reAllocations))
|
||||||
|
return reAllocations, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func formatNodeIDs(ids []int64) []string {
|
func formatNodeIDs(ids []int64) []string {
|
||||||
|
|||||||
@ -20,10 +20,7 @@ import (
|
|||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/golang/protobuf/proto"
|
|
||||||
"github.com/milvus-io/milvus/internal/kv"
|
|
||||||
memkv "github.com/milvus-io/milvus/internal/kv/mem"
|
memkv "github.com/milvus-io/milvus/internal/kv/mem"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"stathat.com/c/consistent"
|
"stathat.com/c/consistent"
|
||||||
)
|
)
|
||||||
@ -321,7 +318,7 @@ func TestConsistentHashDeregisterPolicy(t *testing.T) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestAverageReassignPolicy(t *testing.T) {
|
func TestRoundRobinReassignPolicy(t *testing.T) {
|
||||||
type args struct {
|
type args struct {
|
||||||
store ROChannelStore
|
store ROChannelStore
|
||||||
reassigns []*NodeChannelInfo
|
reassigns []*NodeChannelInfo
|
||||||
@ -342,7 +339,7 @@ func TestAverageReassignPolicy(t *testing.T) {
|
|||||||
},
|
},
|
||||||
[]*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}}}},
|
[]*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}}}},
|
||||||
},
|
},
|
||||||
nil,
|
[]*ChannelOp{},
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"test normal reassing",
|
"test normal reassing",
|
||||||
@ -361,35 +358,18 @@ func TestAverageReassignPolicy(t *testing.T) {
|
|||||||
}
|
}
|
||||||
for _, tt := range tests {
|
for _, tt := range tests {
|
||||||
t.Run(tt.name, func(t *testing.T) {
|
t.Run(tt.name, func(t *testing.T) {
|
||||||
got := AverageReassignPolicy(tt.args.store, tt.args.reassigns)
|
got := RoundRobinReassignPolicy(tt.args.store, tt.args.reassigns)
|
||||||
assert.EqualValues(t, tt.want, got)
|
assert.EqualValues(t, tt.want, got)
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func TestBgCheckWithMaxWatchDuration(t *testing.T) {
|
func TestBgCheckForChannelBalance(t *testing.T) {
|
||||||
type watch struct {
|
|
||||||
nodeID int64
|
|
||||||
name string
|
|
||||||
info *datapb.ChannelWatchInfo
|
|
||||||
}
|
|
||||||
getKv := func(watchInfos []*watch) kv.TxnKV {
|
|
||||||
kv := memkv.NewMemoryKV()
|
|
||||||
for _, info := range watchInfos {
|
|
||||||
k := buildNodeChannelKey(info.nodeID, info.name)
|
|
||||||
v, _ := proto.Marshal(info.info)
|
|
||||||
kv.Save(k, string(v))
|
|
||||||
}
|
|
||||||
return kv
|
|
||||||
}
|
|
||||||
|
|
||||||
type args struct {
|
type args struct {
|
||||||
kv kv.TxnKV
|
|
||||||
channels []*NodeChannelInfo
|
channels []*NodeChannelInfo
|
||||||
timestamp time.Time
|
timestamp time.Time
|
||||||
}
|
}
|
||||||
|
|
||||||
ts := time.Now()
|
|
||||||
tests := []struct {
|
tests := []struct {
|
||||||
name string
|
name string
|
||||||
args args
|
args args
|
||||||
@ -397,30 +377,50 @@ func TestBgCheckWithMaxWatchDuration(t *testing.T) {
|
|||||||
wantErr error
|
wantErr error
|
||||||
}{
|
}{
|
||||||
{
|
{
|
||||||
"test normal expiration",
|
"test even distribution",
|
||||||
args{
|
args{
|
||||||
getKv([]*watch{{1, "chan1", &datapb.ChannelWatchInfo{StartTs: ts.Unix(), State: datapb.ChannelWatchState_Uncomplete}},
|
[]*NodeChannelInfo{
|
||||||
{1, "chan2", &datapb.ChannelWatchInfo{StartTs: ts.Unix(), State: datapb.ChannelWatchState_Complete}}}),
|
{1, []*channel{{Name: "chan1", CollectionID: 1}, {Name: "chan2", CollectionID: 1}}},
|
||||||
[]*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}, {Name: "chan2", CollectionID: 1}}}},
|
{2, []*channel{{Name: "chan1", CollectionID: 2}, {Name: "chan2", CollectionID: 2}}},
|
||||||
ts.Add(Params.DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second)),
|
{3, []*channel{{Name: "chan1", CollectionID: 3}, {Name: "chan2", CollectionID: 3}}},
|
||||||
|
},
|
||||||
|
time.Now(),
|
||||||
},
|
},
|
||||||
[]*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}}}},
|
//there should be no reallocate
|
||||||
|
[]*NodeChannelInfo{},
|
||||||
nil,
|
nil,
|
||||||
},
|
},
|
||||||
{
|
{
|
||||||
"test no expiration",
|
"test uneven with conservative effect",
|
||||||
args{
|
args{
|
||||||
getKv([]*watch{{1, "chan1", &datapb.ChannelWatchInfo{StartTs: ts.Unix(), State: datapb.ChannelWatchState_Uncomplete}}}),
|
[]*NodeChannelInfo{
|
||||||
[]*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}}}},
|
{1, []*channel{{Name: "chan1", CollectionID: 1}, {Name: "chan2", CollectionID: 1}}},
|
||||||
ts.Add(Params.DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second)).Add(-time.Second),
|
{2, []*channel{}},
|
||||||
|
},
|
||||||
|
time.Now(),
|
||||||
},
|
},
|
||||||
|
// as we deem that the node having only one channel more than average as even, so there's no reallocation
|
||||||
|
// for this test case
|
||||||
[]*NodeChannelInfo{},
|
[]*NodeChannelInfo{},
|
||||||
nil,
|
nil,
|
||||||
},
|
},
|
||||||
|
{
|
||||||
|
"test uneven with zero",
|
||||||
|
args{
|
||||||
|
[]*NodeChannelInfo{
|
||||||
|
{1, []*channel{{Name: "chan1", CollectionID: 1}, {Name: "chan2", CollectionID: 1},
|
||||||
|
{Name: "chan3", CollectionID: 1}}},
|
||||||
|
{2, []*channel{}},
|
||||||
|
},
|
||||||
|
time.Now(),
|
||||||
|
},
|
||||||
|
[]*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}}}},
|
||||||
|
nil,
|
||||||
|
},
|
||||||
}
|
}
|
||||||
for _, tt := range tests {
|
for _, tt := range tests {
|
||||||
t.Run(tt.name, func(t *testing.T) {
|
t.Run(tt.name, func(t *testing.T) {
|
||||||
policy := BgCheckWithMaxWatchDuration(tt.args.kv)
|
policy := BgBalanceCheck
|
||||||
got, err := policy(tt.args.channels, tt.args.timestamp)
|
got, err := policy(tt.args.channels, tt.args.timestamp)
|
||||||
assert.Equal(t, tt.wantErr, err)
|
assert.Equal(t, tt.wantErr, err)
|
||||||
assert.EqualValues(t, tt.want, got)
|
assert.EqualValues(t, tt.want, got)
|
||||||
@ -428,6 +428,208 @@ func TestBgCheckWithMaxWatchDuration(t *testing.T) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func TestAvgReassignPolicy(t *testing.T) {
|
||||||
|
type args struct {
|
||||||
|
store ROChannelStore
|
||||||
|
reassigns []*NodeChannelInfo
|
||||||
|
}
|
||||||
|
tests := []struct {
|
||||||
|
name string
|
||||||
|
args args
|
||||||
|
want ChannelOpSet
|
||||||
|
}{
|
||||||
|
{
|
||||||
|
"test_only_one_node",
|
||||||
|
args{
|
||||||
|
&ChannelStore{
|
||||||
|
memkv.NewMemoryKV(),
|
||||||
|
map[int64]*NodeChannelInfo{
|
||||||
|
1: {1, []*channel{{Name: "chan1", CollectionID: 1}}},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
[]*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}}}},
|
||||||
|
},
|
||||||
|
//as there's no available nodes except the input node, there's no reassign plan generated
|
||||||
|
[]*ChannelOp{},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"test_zero_avg",
|
||||||
|
args{
|
||||||
|
&ChannelStore{
|
||||||
|
memkv.NewMemoryKV(),
|
||||||
|
map[int64]*NodeChannelInfo{
|
||||||
|
1: {1, []*channel{{Name: "chan1", CollectionID: 1}}},
|
||||||
|
2: {2, []*channel{}},
|
||||||
|
3: {2, []*channel{}},
|
||||||
|
4: {2, []*channel{}},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
[]*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}}}},
|
||||||
|
},
|
||||||
|
[]*ChannelOp{
|
||||||
|
//as we use ceil to calculate the wanted average number, there should be one reassign
|
||||||
|
//though the average num less than 1
|
||||||
|
{Delete, 1, []*channel{{Name: "chan1", CollectionID: 1}}, nil},
|
||||||
|
{Add, 2, []*channel{{Name: "chan1", CollectionID: 1}}, nil}},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"test_normal_reassigning_for_one_available_nodes",
|
||||||
|
args{
|
||||||
|
&ChannelStore{
|
||||||
|
memkv.NewMemoryKV(),
|
||||||
|
map[int64]*NodeChannelInfo{
|
||||||
|
1: {1, []*channel{{Name: "chan1", CollectionID: 1}, {Name: "chan2", CollectionID: 1}}},
|
||||||
|
2: {2, []*channel{}},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
[]*NodeChannelInfo{{1, []*channel{{Name: "chan1", CollectionID: 1}, {Name: "chan2", CollectionID: 1}}}},
|
||||||
|
},
|
||||||
|
[]*ChannelOp{
|
||||||
|
{Delete, 1, []*channel{{Name: "chan1", CollectionID: 1}, {Name: "chan2", CollectionID: 1}}, nil},
|
||||||
|
{Add, 2, []*channel{{Name: "chan1", CollectionID: 1}, {Name: "chan2", CollectionID: 1}}, nil}},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"test_normal_reassigning_for_multiple_available_nodes",
|
||||||
|
args{
|
||||||
|
&ChannelStore{
|
||||||
|
memkv.NewMemoryKV(),
|
||||||
|
map[int64]*NodeChannelInfo{
|
||||||
|
1: {1, []*channel{
|
||||||
|
{Name: "chan1", CollectionID: 1},
|
||||||
|
{Name: "chan2", CollectionID: 1},
|
||||||
|
{Name: "chan3", CollectionID: 1},
|
||||||
|
{Name: "chan4", CollectionID: 1}}},
|
||||||
|
2: {2, []*channel{}},
|
||||||
|
3: {3, []*channel{}},
|
||||||
|
4: {4, []*channel{}},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
[]*NodeChannelInfo{{1, []*channel{
|
||||||
|
{Name: "chan1", CollectionID: 1},
|
||||||
|
{Name: "chan2", CollectionID: 1},
|
||||||
|
{Name: "chan3", CollectionID: 1},
|
||||||
|
}}},
|
||||||
|
},
|
||||||
|
[]*ChannelOp{
|
||||||
|
{Delete, 1, []*channel{
|
||||||
|
{Name: "chan1", CollectionID: 1},
|
||||||
|
{Name: "chan2", CollectionID: 1},
|
||||||
|
{Name: "chan3", CollectionID: 1}},
|
||||||
|
nil},
|
||||||
|
{Add, 2, []*channel{{Name: "chan1", CollectionID: 1}}, nil},
|
||||||
|
{Add, 3, []*channel{{Name: "chan2", CollectionID: 1}}, nil},
|
||||||
|
{Add, 4, []*channel{{Name: "chan3", CollectionID: 1}}, nil},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"test_reassigning_for_extreme_case",
|
||||||
|
args{
|
||||||
|
&ChannelStore{
|
||||||
|
memkv.NewMemoryKV(),
|
||||||
|
map[int64]*NodeChannelInfo{
|
||||||
|
1: {1, []*channel{
|
||||||
|
{Name: "chan1", CollectionID: 1}, {Name: "chan2", CollectionID: 1},
|
||||||
|
{Name: "chan3", CollectionID: 1}, {Name: "chan4", CollectionID: 1},
|
||||||
|
{Name: "chan5", CollectionID: 1}, {Name: "chan6", CollectionID: 1},
|
||||||
|
{Name: "chan7", CollectionID: 1}, {Name: "chan8", CollectionID: 1},
|
||||||
|
{Name: "chan9", CollectionID: 1}, {Name: "chan10", CollectionID: 1},
|
||||||
|
{Name: "chan11", CollectionID: 1}, {Name: "chan12", CollectionID: 1},
|
||||||
|
}},
|
||||||
|
2: {2, []*channel{
|
||||||
|
{Name: "chan13", CollectionID: 1}, {Name: "chan14", CollectionID: 1},
|
||||||
|
}},
|
||||||
|
3: {3, []*channel{{Name: "chan15", CollectionID: 1}}},
|
||||||
|
4: {4, []*channel{}},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
[]*NodeChannelInfo{{1, []*channel{
|
||||||
|
{Name: "chan1", CollectionID: 1}, {Name: "chan2", CollectionID: 1},
|
||||||
|
{Name: "chan3", CollectionID: 1}, {Name: "chan4", CollectionID: 1},
|
||||||
|
{Name: "chan5", CollectionID: 1}, {Name: "chan6", CollectionID: 1},
|
||||||
|
{Name: "chan7", CollectionID: 1}, {Name: "chan8", CollectionID: 1},
|
||||||
|
{Name: "chan9", CollectionID: 1}, {Name: "chan10", CollectionID: 1},
|
||||||
|
{Name: "chan11", CollectionID: 1}, {Name: "chan12", CollectionID: 1},
|
||||||
|
}}},
|
||||||
|
},
|
||||||
|
[]*ChannelOp{
|
||||||
|
{Delete, 1, []*channel{
|
||||||
|
{Name: "chan1", CollectionID: 1}, {Name: "chan2", CollectionID: 1},
|
||||||
|
{Name: "chan3", CollectionID: 1}, {Name: "chan4", CollectionID: 1},
|
||||||
|
{Name: "chan5", CollectionID: 1}, {Name: "chan6", CollectionID: 1},
|
||||||
|
{Name: "chan7", CollectionID: 1}, {Name: "chan8", CollectionID: 1},
|
||||||
|
{Name: "chan9", CollectionID: 1}, {Name: "chan10", CollectionID: 1},
|
||||||
|
{Name: "chan11", CollectionID: 1}, {Name: "chan12", CollectionID: 1},
|
||||||
|
}, nil},
|
||||||
|
{Add, 4, []*channel{
|
||||||
|
{Name: "chan1", CollectionID: 1}, {Name: "chan2", CollectionID: 1},
|
||||||
|
{Name: "chan3", CollectionID: 1}, {Name: "chan4", CollectionID: 1},
|
||||||
|
{Name: "chan5", CollectionID: 1}}, nil},
|
||||||
|
{Add, 3, []*channel{
|
||||||
|
{Name: "chan6", CollectionID: 1}, {Name: "chan7", CollectionID: 1},
|
||||||
|
{Name: "chan8", CollectionID: 1}, {Name: "chan9", CollectionID: 1},
|
||||||
|
}, nil},
|
||||||
|
{Add, 2, []*channel{
|
||||||
|
{Name: "chan10", CollectionID: 1}, {Name: "chan11", CollectionID: 1},
|
||||||
|
{Name: "chan12", CollectionID: 1},
|
||||||
|
}, nil},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
for _, tt := range tests {
|
||||||
|
if tt.name == "test_reassigning_for_extreme_case" ||
|
||||||
|
tt.name == "test_normal_reassigning_for_multiple_available_nodes" {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
t.Run(tt.name, func(t *testing.T) {
|
||||||
|
got := AverageReassignPolicy(tt.args.store, tt.args.reassigns)
|
||||||
|
assert.EqualValues(t, tt.want, got)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
func TestAvgBalanceChannelPolicy(t *testing.T) {
|
||||||
|
type args struct {
|
||||||
|
store ROChannelStore
|
||||||
|
}
|
||||||
|
tests := []struct {
|
||||||
|
name string
|
||||||
|
args args
|
||||||
|
want ChannelOpSet
|
||||||
|
}{
|
||||||
|
{
|
||||||
|
"test_only_one_node",
|
||||||
|
args{
|
||||||
|
&ChannelStore{
|
||||||
|
memkv.NewMemoryKV(),
|
||||||
|
map[int64]*NodeChannelInfo{
|
||||||
|
1: {
|
||||||
|
1, []*channel{
|
||||||
|
{Name: "chan1", CollectionID: 1},
|
||||||
|
{Name: "chan2", CollectionID: 1},
|
||||||
|
{Name: "chan3", CollectionID: 1},
|
||||||
|
{Name: "chan4", CollectionID: 1},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
2: {2, []*channel{}},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
[]*ChannelOp{
|
||||||
|
{Add, 1, []*channel{
|
||||||
|
{Name: "chan1", CollectionID: 1},
|
||||||
|
}, nil},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, tt := range tests {
|
||||||
|
t.Run(tt.name, func(t *testing.T) {
|
||||||
|
got := AvgBalanceChannelPolicy(tt.args.store, time.Now())
|
||||||
|
assert.EqualValues(t, tt.want, got)
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
func TestAvgAssignRegisterPolicy(t *testing.T) {
|
func TestAvgAssignRegisterPolicy(t *testing.T) {
|
||||||
type args struct {
|
type args struct {
|
||||||
store ROChannelStore
|
store ROChannelStore
|
||||||
|
|||||||
@ -391,7 +391,8 @@ func (s *Server) initCluster() error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
var err error
|
var err error
|
||||||
s.channelManager, err = NewChannelManager(s.kvClient, s.handler, withMsgstreamFactory(s.factory), withStateChecker())
|
s.channelManager, err = NewChannelManager(s.kvClient, s.handler, withMsgstreamFactory(s.factory),
|
||||||
|
withStateChecker(), withBgChecker())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|||||||
@ -1622,7 +1622,9 @@ Max read concurrency must greater than or equal to 1, and less than or equal to
|
|||||||
type dataCoordConfig struct {
|
type dataCoordConfig struct {
|
||||||
|
|
||||||
// --- CHANNEL ---
|
// --- CHANNEL ---
|
||||||
WatchTimeoutInterval ParamItem `refreshable:"false"`
|
WatchTimeoutInterval ParamItem `refreshable:"false"`
|
||||||
|
ChannelBalanceSilentDuration ParamItem `refreshable:"true"`
|
||||||
|
ChannelBalanceInterval ParamItem `refreshable:"true"`
|
||||||
|
|
||||||
// --- SEGMENTS ---
|
// --- SEGMENTS ---
|
||||||
SegmentMaxSize ParamItem `refreshable:"false"`
|
SegmentMaxSize ParamItem `refreshable:"false"`
|
||||||
@ -1677,6 +1679,24 @@ func (p *dataCoordConfig) init(base *BaseTable) {
|
|||||||
}
|
}
|
||||||
p.WatchTimeoutInterval.Init(base.mgr)
|
p.WatchTimeoutInterval.Init(base.mgr)
|
||||||
|
|
||||||
|
p.ChannelBalanceSilentDuration = ParamItem{
|
||||||
|
Key: "dataCoord.channel.balanceSilentDuration",
|
||||||
|
Version: "2.2.3",
|
||||||
|
DefaultValue: "300",
|
||||||
|
Doc: "The duration after which the channel manager start background channel balancing",
|
||||||
|
Export: true,
|
||||||
|
}
|
||||||
|
p.ChannelBalanceSilentDuration.Init(base.mgr)
|
||||||
|
|
||||||
|
p.ChannelBalanceInterval = ParamItem{
|
||||||
|
Key: "dataCoord.channel.balanceInterval",
|
||||||
|
Version: "2.2.3",
|
||||||
|
DefaultValue: "360",
|
||||||
|
Doc: "The interval with which the channel manager check dml channel balance status",
|
||||||
|
Export: true,
|
||||||
|
}
|
||||||
|
p.ChannelBalanceInterval.Init(base.mgr)
|
||||||
|
|
||||||
p.SegmentMaxSize = ParamItem{
|
p.SegmentMaxSize = ParamItem{
|
||||||
Key: "dataCoord.segment.maxSize",
|
Key: "dataCoord.segment.maxSize",
|
||||||
Version: "2.0.0",
|
Version: "2.0.0",
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user