mirror of
https://gitee.com/milvus-io/milvus.git
synced 2024-11-29 18:38:44 +08:00
enhance: make subfunctions of datanode component modular (#33992)
issue: #33994 also remove deprecated channel manager based on the etcd implementation Signed-off-by: jaime <yun.zhang@zilliz.com>
This commit is contained in:
parent
14e827dc6c
commit
d6afb31b94
4
Makefile
4
Makefile
@ -484,8 +484,8 @@ generate-mockery-datanode: getdeps
|
||||
$(INSTALL_PATH)/mockery --name=WriteBuffer --dir=$(PWD)/internal/datanode/writebuffer --output=$(PWD)/internal/datanode/writebuffer --filename=mock_write_buffer.go --with-expecter --structname=MockWriteBuffer --outpkg=writebuffer --inpackage
|
||||
$(INSTALL_PATH)/mockery --name=BufferManager --dir=$(PWD)/internal/datanode/writebuffer --output=$(PWD)/internal/datanode/writebuffer --filename=mock_mananger.go --with-expecter --structname=MockBufferManager --outpkg=writebuffer --inpackage
|
||||
$(INSTALL_PATH)/mockery --name=BinlogIO --dir=$(PWD)/internal/datanode/io --output=$(PWD)/internal/datanode/io --filename=mock_binlogio.go --with-expecter --structname=MockBinlogIO --outpkg=io --inpackage
|
||||
$(INSTALL_PATH)/mockery --name=FlowgraphManager --dir=$(PWD)/internal/datanode --output=$(PWD)/internal/datanode --filename=mock_fgmanager.go --with-expecter --structname=MockFlowgraphManager --outpkg=datanode --inpackage
|
||||
$(INSTALL_PATH)/mockery --name=ChannelManager --dir=$(PWD)/internal/datanode --output=$(PWD)/internal/datanode --filename=mock_channelmanager.go --with-expecter --structname=MockChannelManager --outpkg=datanode --inpackage
|
||||
$(INSTALL_PATH)/mockery --name=FlowgraphManager --dir=$(PWD)/internal/datanode/pipeline --output=$(PWD)/internal/datanode/pipeline --filename=mock_fgmanager.go --with-expecter --structname=MockFlowgraphManager --outpkg=pipeline --inpackage
|
||||
$(INSTALL_PATH)/mockery --name=ChannelManager --dir=$(PWD)/internal/datanode/channel --output=$(PWD)/internal/datanode/channel --filename=mock_channelmanager.go --with-expecter --structname=MockChannelManager --outpkg=channel --inpackage
|
||||
$(INSTALL_PATH)/mockery --name=Compactor --dir=$(PWD)/internal/datanode/compaction --output=$(PWD)/internal/datanode/compaction --filename=mock_compactor.go --with-expecter --structname=MockCompactor --outpkg=compaction --inpackage
|
||||
|
||||
generate-mockery-metastore: getdeps
|
||||
|
@ -419,7 +419,6 @@ indexNode:
|
||||
dataCoord:
|
||||
channel:
|
||||
watchTimeoutInterval: 300 # Timeout on watching channels (in seconds). Datanode tickler update watch progress will reset timeout timer.
|
||||
balanceWithRpc: true # Whether to enable balance with RPC, default to use etcd watch
|
||||
legacyVersionWithoutRPCWatch: 2.4.1 # Datanodes <= this version are considered as legacy nodes, which doesn't have rpc based watch(). This is only used during rolling upgrade where legacy nodes won't get new channels
|
||||
balanceSilentDuration: 300 # The duration after which the channel manager start background channel balancing
|
||||
balanceInterval: 360 # The interval with which the channel manager check dml channel balance status
|
||||
|
@ -26,7 +26,6 @@ import (
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
)
|
||||
|
||||
type ROChannel interface {
|
||||
@ -50,17 +49,7 @@ func NewRWChannel(name string,
|
||||
schema *schemapb.CollectionSchema,
|
||||
createTs uint64,
|
||||
) RWChannel {
|
||||
if paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.GetAsBool() {
|
||||
return &StateChannel{
|
||||
Name: name,
|
||||
CollectionID: collectionID,
|
||||
StartPositions: startPos,
|
||||
Schema: schema,
|
||||
CreateTimestamp: createTs,
|
||||
}
|
||||
}
|
||||
|
||||
return &channelMeta{
|
||||
return &StateChannel{
|
||||
Name: name,
|
||||
CollectionID: collectionID,
|
||||
StartPositions: startPos,
|
||||
|
@ -1,225 +0,0 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datacoord
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"path"
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/atomic"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/kv"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
type channelStateTimer struct {
|
||||
watchkv kv.WatchKV
|
||||
|
||||
runningTimers *typeutil.ConcurrentMap[string, *time.Timer]
|
||||
runningTimerStops *typeutil.ConcurrentMap[string, chan struct{}] // channel name to timer stop channels
|
||||
|
||||
etcdWatcher clientv3.WatchChan
|
||||
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.WatchKV) *channelStateTimer {
|
||||
return &channelStateTimer{
|
||||
watchkv: kv,
|
||||
timeoutWatcher: make(chan *ackEvent, 20),
|
||||
runningTimers: typeutil.NewConcurrentMap[string, *time.Timer](),
|
||||
runningTimerStops: typeutil.NewConcurrentMap[string, chan struct{}](),
|
||||
}
|
||||
}
|
||||
|
||||
func (c *channelStateTimer) getWatchers(prefix string) (clientv3.WatchChan, chan *ackEvent) {
|
||||
if c.etcdWatcher == nil {
|
||||
c.etcdWatcher = c.watchkv.WatchWithPrefix(prefix)
|
||||
}
|
||||
return c.etcdWatcher, c.timeoutWatcher
|
||||
}
|
||||
|
||||
func (c *channelStateTimer) getWatchersWithRevision(prefix string, revision int64) (clientv3.WatchChan, chan *ackEvent) {
|
||||
c.etcdWatcher = c.watchkv.WatchWithRevision(prefix, revision)
|
||||
return c.etcdWatcher, c.timeoutWatcher
|
||||
}
|
||||
|
||||
func (c *channelStateTimer) loadAllChannels(nodeID UniqueID) ([]*datapb.ChannelWatchInfo, error) {
|
||||
prefix := path.Join(Params.CommonCfg.DataCoordWatchSubPath.GetValue(), strconv.FormatInt(nodeID, 10))
|
||||
|
||||
// TODO: change to LoadWithPrefixBytes
|
||||
keys, values, err := c.watchkv.LoadWithPrefix(prefix)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var ret []*datapb.ChannelWatchInfo
|
||||
|
||||
for i, k := range keys {
|
||||
watchInfo, err := parseWatchInfo(k, []byte(values[i]))
|
||||
if err != nil {
|
||||
// TODO: delete this kv later
|
||||
log.Warn("invalid watchInfo loaded", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
ret = append(ret, watchInfo)
|
||||
}
|
||||
|
||||
return ret, nil
|
||||
}
|
||||
|
||||
// startOne can write ToWatch or ToRelease states.
|
||||
func (c *channelStateTimer) startOne(watchState datapb.ChannelWatchState, channelName string, nodeID UniqueID, timeout time.Duration) {
|
||||
if timeout == 0 {
|
||||
log.Info("zero timeoutTs, skip starting timer",
|
||||
zap.String("watch state", watchState.String()),
|
||||
zap.Int64("nodeID", nodeID),
|
||||
zap.String("channelName", channelName),
|
||||
)
|
||||
return
|
||||
}
|
||||
|
||||
stop := make(chan struct{})
|
||||
ticker := time.NewTimer(timeout)
|
||||
c.removeTimers([]string{channelName})
|
||||
c.runningTimerStops.Insert(channelName, stop)
|
||||
c.runningTimers.Insert(channelName, ticker)
|
||||
c.runningTimerCount.Inc()
|
||||
go func() {
|
||||
log.Info("timer started",
|
||||
zap.String("watch state", watchState.String()),
|
||||
zap.Int64("nodeID", nodeID),
|
||||
zap.String("channelName", channelName),
|
||||
zap.Duration("check interval", timeout))
|
||||
defer ticker.Stop()
|
||||
|
||||
select {
|
||||
case <-ticker.C:
|
||||
// check tickle at path as :tickle/[prefix]/{channel_name}
|
||||
c.removeTimers([]string{channelName})
|
||||
log.Warn("timeout and stop timer: wait for channel ACK timeout",
|
||||
zap.String("watch state", watchState.String()),
|
||||
zap.Int64("nodeID", nodeID),
|
||||
zap.String("channelName", channelName),
|
||||
zap.Duration("timeout interval", timeout),
|
||||
zap.Int32("runningTimerCount", c.runningTimerCount.Load()))
|
||||
ackType := getAckType(watchState)
|
||||
c.notifyTimeoutWatcher(&ackEvent{ackType, channelName, nodeID})
|
||||
return
|
||||
case <-stop:
|
||||
log.Info("stop timer before timeout",
|
||||
zap.String("watch state", watchState.String()),
|
||||
zap.Int64("nodeID", nodeID),
|
||||
zap.String("channelName", channelName),
|
||||
zap.Duration("timeout interval", timeout),
|
||||
zap.Int32("runningTimerCount", c.runningTimerCount.Load()))
|
||||
return
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
func (c *channelStateTimer) notifyTimeoutWatcher(e *ackEvent) {
|
||||
c.timeoutWatcher <- e
|
||||
}
|
||||
|
||||
func (c *channelStateTimer) removeTimers(channels []string) {
|
||||
for _, channel := range channels {
|
||||
if stop, ok := c.runningTimerStops.GetAndRemove(channel); ok {
|
||||
close(stop)
|
||||
c.runningTimers.GetAndRemove(channel)
|
||||
c.runningTimerCount.Dec()
|
||||
log.Info("remove timer for channel", zap.String("channel", channel),
|
||||
zap.Int32("timerCount", c.runningTimerCount.Load()))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *channelStateTimer) stopIfExist(e *ackEvent) {
|
||||
stop, ok := c.runningTimerStops.GetAndRemove(e.channelName)
|
||||
if ok && e.ackType != watchTimeoutAck && e.ackType != releaseTimeoutAck {
|
||||
close(stop)
|
||||
c.runningTimers.GetAndRemove(e.channelName)
|
||||
c.runningTimerCount.Dec()
|
||||
log.Info("stop timer for channel", zap.String("channel", e.channelName),
|
||||
zap.Int32("timerCount", c.runningTimerCount.Load()))
|
||||
}
|
||||
}
|
||||
|
||||
func (c *channelStateTimer) resetIfExist(channel string, interval time.Duration) {
|
||||
if timer, ok := c.runningTimers.Get(channel); ok {
|
||||
timer.Reset(interval)
|
||||
}
|
||||
}
|
||||
|
||||
// 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) {
|
||||
watchInfo := datapb.ChannelWatchInfo{}
|
||||
if err := proto.Unmarshal(data, &watchInfo); err != nil {
|
||||
return nil, fmt.Errorf("invalid event data: fail to parse ChannelWatchInfo, key: %s, err: %v", key, err)
|
||||
}
|
||||
|
||||
if watchInfo.Vchan == nil {
|
||||
return nil, fmt.Errorf("invalid event: ChannelWatchInfo with nil VChannelInfo, key: %s", key)
|
||||
}
|
||||
reviseVChannelInfo(watchInfo.GetVchan())
|
||||
|
||||
return &watchInfo, nil
|
||||
}
|
||||
|
||||
// parseAckEvent transfers key-values from etcd into ackEvent
|
||||
func parseAckEvent(nodeID UniqueID, info *datapb.ChannelWatchInfo) *ackEvent {
|
||||
ret := &ackEvent{
|
||||
ackType: getAckType(info.GetState()),
|
||||
channelName: info.GetVchan().GetChannelName(),
|
||||
nodeID: nodeID,
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
func getAckType(state datapb.ChannelWatchState) ackType {
|
||||
switch state {
|
||||
case datapb.ChannelWatchState_WatchSuccess, datapb.ChannelWatchState_Complete:
|
||||
return watchSuccessAck
|
||||
case datapb.ChannelWatchState_WatchFailure:
|
||||
return watchFailAck
|
||||
case datapb.ChannelWatchState_ReleaseSuccess:
|
||||
return releaseSuccessAck
|
||||
case datapb.ChannelWatchState_ReleaseFailure:
|
||||
return releaseFailAck
|
||||
case datapb.ChannelWatchState_ToWatch, datapb.ChannelWatchState_Uncomplete: // unchange watch states generates timeout acks
|
||||
return watchTimeoutAck
|
||||
case datapb.ChannelWatchState_ToRelease: // unchange watch states generates timeout acks
|
||||
return releaseTimeoutAck
|
||||
default:
|
||||
return invalidAck
|
||||
}
|
||||
}
|
@ -1,246 +0,0 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datacoord
|
||||
|
||||
import (
|
||||
"path"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
)
|
||||
|
||||
func TestChannelStateTimer(t *testing.T) {
|
||||
kv := getWatchKV(t)
|
||||
defer kv.Close()
|
||||
|
||||
prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue()
|
||||
|
||||
t.Run("test getWatcher", func(t *testing.T) {
|
||||
timer := newChannelStateTimer(kv)
|
||||
|
||||
etcdCh, timeoutCh := timer.getWatchers(prefix)
|
||||
assert.NotNil(t, etcdCh)
|
||||
assert.NotNil(t, timeoutCh)
|
||||
|
||||
timer.getWatchers(prefix)
|
||||
assert.NotNil(t, etcdCh)
|
||||
assert.NotNil(t, timeoutCh)
|
||||
})
|
||||
|
||||
t.Run("test loadAllChannels", func(t *testing.T) {
|
||||
defer kv.RemoveWithPrefix("")
|
||||
timer := newChannelStateTimer(kv)
|
||||
timer.loadAllChannels(1)
|
||||
|
||||
validWatchInfo := datapb.ChannelWatchInfo{
|
||||
Vchan: &datapb.VchannelInfo{},
|
||||
StartTs: time.Now().Unix(),
|
||||
State: datapb.ChannelWatchState_ToWatch,
|
||||
}
|
||||
validData, err := proto.Marshal(&validWatchInfo)
|
||||
require.NoError(t, err)
|
||||
|
||||
prefix = Params.CommonCfg.DataCoordWatchSubPath.GetValue()
|
||||
prepareKvs := map[string]string{
|
||||
path.Join(prefix, "1/channel-1"): "invalidWatchInfo",
|
||||
path.Join(prefix, "1/channel-2"): string(validData),
|
||||
path.Join(prefix, "2/channel-3"): string(validData),
|
||||
}
|
||||
|
||||
err = kv.MultiSave(prepareKvs)
|
||||
require.NoError(t, err)
|
||||
|
||||
tests := []struct {
|
||||
inNodeID UniqueID
|
||||
outLen int
|
||||
}{
|
||||
{1, 1},
|
||||
{2, 1},
|
||||
{3, 0},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
infos, err := timer.loadAllChannels(test.inNodeID)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, test.outLen, len(infos))
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("test startOne", func(t *testing.T) {
|
||||
normalTimeoutTs := 20 * time.Second
|
||||
nowTimeoutTs := 1 * time.Millisecond
|
||||
zeroTimeoutTs := 0 * time.Second
|
||||
resetTimeoutTs := 30 * time.Second
|
||||
tests := []struct {
|
||||
channelName string
|
||||
timeoutTs time.Duration
|
||||
|
||||
description string
|
||||
}{
|
||||
{"channel-1", normalTimeoutTs, "test stop"},
|
||||
{"channel-2", nowTimeoutTs, "test timeout"},
|
||||
{"channel-3", zeroTimeoutTs, "not start"},
|
||||
{"channel-4", resetTimeoutTs, "reset timer"},
|
||||
}
|
||||
|
||||
timer := newChannelStateTimer(kv)
|
||||
|
||||
_, timeoutCh := timer.getWatchers(prefix)
|
||||
|
||||
for _, test := range tests {
|
||||
t.Run(test.description, func(t *testing.T) {
|
||||
timer.startOne(datapb.ChannelWatchState_ToWatch, test.channelName, 1, test.timeoutTs)
|
||||
if test.timeoutTs == nowTimeoutTs {
|
||||
e := <-timeoutCh
|
||||
assert.Equal(t, watchTimeoutAck, e.ackType)
|
||||
assert.Equal(t, test.channelName, e.channelName)
|
||||
} else if test.timeoutTs == resetTimeoutTs {
|
||||
timer.resetIfExist(test.channelName, nowTimeoutTs)
|
||||
e := <-timeoutCh
|
||||
assert.Equal(t, watchTimeoutAck, e.ackType)
|
||||
assert.Equal(t, test.channelName, e.channelName)
|
||||
} else {
|
||||
timer.stopIfExist(&ackEvent{watchSuccessAck, test.channelName, 1})
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
timer.startOne(datapb.ChannelWatchState_ToWatch, "channel-remove", 1, normalTimeoutTs)
|
||||
timer.removeTimers([]string{"channel-remove"})
|
||||
})
|
||||
|
||||
t.Run("test startOne no leaking issue 17335", func(t *testing.T) {
|
||||
timer := newChannelStateTimer(kv)
|
||||
|
||||
timer.startOne(datapb.ChannelWatchState_ToRelease, "channel-1", 1, 20*time.Second)
|
||||
stop, ok := timer.runningTimerStops.Get("channel-1")
|
||||
require.True(t, ok)
|
||||
|
||||
timer.startOne(datapb.ChannelWatchState_ToWatch, "channel-1", 1, 20*time.Second)
|
||||
_, ok = <-stop
|
||||
assert.False(t, ok)
|
||||
|
||||
stop2, ok := timer.runningTimerStops.Get("channel-1")
|
||||
assert.True(t, ok)
|
||||
|
||||
timer.removeTimers([]string{"channel-1"})
|
||||
_, ok = <-stop2
|
||||
assert.False(t, ok)
|
||||
})
|
||||
}
|
||||
|
||||
func TestChannelStateTimer_parses(t *testing.T) {
|
||||
const (
|
||||
ValidTest = true
|
||||
InValidTest = false
|
||||
)
|
||||
|
||||
t.Run("test parseWatchInfo", func(t *testing.T) {
|
||||
validWatchInfo := datapb.ChannelWatchInfo{
|
||||
Vchan: &datapb.VchannelInfo{},
|
||||
StartTs: time.Now().Unix(),
|
||||
State: datapb.ChannelWatchState_ToWatch,
|
||||
}
|
||||
validData, err := proto.Marshal(&validWatchInfo)
|
||||
require.NoError(t, err)
|
||||
|
||||
invalidDataUnableToMarshal := []byte("invalidData")
|
||||
|
||||
invalidWatchInfoNilVchan := validWatchInfo
|
||||
invalidWatchInfoNilVchan.Vchan = nil
|
||||
invalidDataNilVchan, err := proto.Marshal(&invalidWatchInfoNilVchan)
|
||||
require.NoError(t, err)
|
||||
|
||||
tests := []struct {
|
||||
inKey string
|
||||
inData []byte
|
||||
|
||||
isValid bool
|
||||
description string
|
||||
}{
|
||||
{"key", validData, ValidTest, "test with valid watchInfo"},
|
||||
{"key", invalidDataUnableToMarshal, InValidTest, "test with watchInfo unable to marshal"},
|
||||
{"key", invalidDataNilVchan, InValidTest, "test with watchInfo with nil Vchan"},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
t.Run(test.description, func(t *testing.T) {
|
||||
info, err := parseWatchInfo(test.inKey, test.inData)
|
||||
if test.isValid {
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, info)
|
||||
assert.Equal(t, info.GetState(), validWatchInfo.GetState())
|
||||
assert.Equal(t, info.GetStartTs(), validWatchInfo.GetStartTs())
|
||||
} else {
|
||||
assert.Nil(t, info)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
})
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("test parseWatchInfo compatibility", func(t *testing.T) {
|
||||
oldWatchInfo := datapb.ChannelWatchInfo{
|
||||
Vchan: &datapb.VchannelInfo{
|
||||
CollectionID: 1,
|
||||
ChannelName: "delta-channel1",
|
||||
UnflushedSegments: []*datapb.SegmentInfo{{ID: 1}},
|
||||
FlushedSegments: []*datapb.SegmentInfo{{ID: 2}},
|
||||
DroppedSegments: []*datapb.SegmentInfo{{ID: 3}},
|
||||
UnflushedSegmentIds: []int64{1},
|
||||
},
|
||||
StartTs: time.Now().Unix(),
|
||||
State: datapb.ChannelWatchState_ToWatch,
|
||||
}
|
||||
|
||||
oldData, err := proto.Marshal(&oldWatchInfo)
|
||||
assert.NoError(t, err)
|
||||
newWatchInfo, err := parseWatchInfo("key", oldData)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, []*datapb.SegmentInfo{}, newWatchInfo.GetVchan().GetUnflushedSegments())
|
||||
assert.Equal(t, []*datapb.SegmentInfo{}, newWatchInfo.GetVchan().GetFlushedSegments())
|
||||
assert.Equal(t, []*datapb.SegmentInfo{}, newWatchInfo.GetVchan().GetDroppedSegments())
|
||||
assert.NotEmpty(t, newWatchInfo.GetVchan().GetUnflushedSegmentIds())
|
||||
assert.NotEmpty(t, newWatchInfo.GetVchan().GetFlushedSegmentIds())
|
||||
assert.NotEmpty(t, newWatchInfo.GetVchan().GetDroppedSegmentIds())
|
||||
})
|
||||
|
||||
t.Run("test getAckType", func(t *testing.T) {
|
||||
tests := []struct {
|
||||
inState datapb.ChannelWatchState
|
||||
outAckType ackType
|
||||
}{
|
||||
{datapb.ChannelWatchState_WatchSuccess, watchSuccessAck},
|
||||
{datapb.ChannelWatchState_WatchFailure, watchFailAck},
|
||||
{datapb.ChannelWatchState_ToWatch, watchTimeoutAck},
|
||||
{datapb.ChannelWatchState_Uncomplete, watchTimeoutAck},
|
||||
{datapb.ChannelWatchState_ReleaseSuccess, releaseSuccessAck},
|
||||
{datapb.ChannelWatchState_ReleaseFailure, releaseFailAck},
|
||||
{datapb.ChannelWatchState_ToRelease, releaseTimeoutAck},
|
||||
{100, invalidAck},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
assert.Equal(t, test.outAckType, getAckType(test.inState))
|
||||
}
|
||||
})
|
||||
}
|
@ -1,903 +0,0 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datacoord
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/samber/lo"
|
||||
v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
|
||||
clientv3 "go.etcd.io/etcd/client/v3"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/kv"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/lock"
|
||||
"github.com/milvus-io/milvus/pkg/util/logutil"
|
||||
)
|
||||
|
||||
// ChannelManagerImpl manages the allocation and the balance between channels and data nodes.
|
||||
type ChannelManagerImpl struct {
|
||||
ctx context.Context
|
||||
mu lock.RWMutex
|
||||
h Handler
|
||||
store RWChannelStore
|
||||
factory ChannelPolicyFactory
|
||||
registerPolicy RegisterPolicy
|
||||
deregisterPolicy DeregisterPolicy
|
||||
assignPolicy ChannelAssignPolicy
|
||||
reassignPolicy ChannelReassignPolicy
|
||||
balancePolicy BalanceChannelPolicy
|
||||
bgChecker ChannelBGChecker
|
||||
msgstreamFactory msgstream.Factory
|
||||
|
||||
stateChecker channelStateChecker
|
||||
stopChecker context.CancelFunc
|
||||
stateTimer *channelStateTimer
|
||||
|
||||
lastActiveTimestamp time.Time
|
||||
}
|
||||
|
||||
// ChannelManagerOpt is to set optional parameters in channel manager.
|
||||
type ChannelManagerOpt func(c *ChannelManagerImpl)
|
||||
|
||||
func withFactory(f ChannelPolicyFactory) ChannelManagerOpt {
|
||||
return func(c *ChannelManagerImpl) { c.factory = f }
|
||||
}
|
||||
|
||||
func withMsgstreamFactory(f msgstream.Factory) ChannelManagerOpt {
|
||||
return func(c *ChannelManagerImpl) { c.msgstreamFactory = f }
|
||||
}
|
||||
|
||||
func withStateChecker() ChannelManagerOpt {
|
||||
return func(c *ChannelManagerImpl) { c.stateChecker = c.watchChannelStatesLoop }
|
||||
}
|
||||
|
||||
func withBgChecker() ChannelManagerOpt {
|
||||
return func(c *ChannelManagerImpl) { c.bgChecker = c.bgCheckChannelsWork }
|
||||
}
|
||||
|
||||
// NewChannelManager creates and returns a new ChannelManager instance.
|
||||
func NewChannelManager(
|
||||
kv kv.WatchKV, // for TxnKv, MetaKv and WatchKV
|
||||
h Handler,
|
||||
options ...ChannelManagerOpt,
|
||||
) (*ChannelManagerImpl, error) {
|
||||
c := &ChannelManagerImpl{
|
||||
ctx: context.TODO(),
|
||||
h: h,
|
||||
factory: NewChannelPolicyFactoryV1(),
|
||||
store: NewChannelStore(kv),
|
||||
stateTimer: newChannelStateTimer(kv),
|
||||
}
|
||||
|
||||
if err := c.store.Reload(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
for _, opt := range options {
|
||||
opt(c)
|
||||
}
|
||||
|
||||
c.registerPolicy = c.factory.NewRegisterPolicy()
|
||||
c.deregisterPolicy = c.factory.NewDeregisterPolicy()
|
||||
c.assignPolicy = c.factory.NewAssignPolicy()
|
||||
c.reassignPolicy = c.factory.NewReassignPolicy()
|
||||
c.balancePolicy = c.factory.NewBalancePolicy()
|
||||
c.lastActiveTimestamp = time.Now()
|
||||
return c, nil
|
||||
}
|
||||
|
||||
// Startup adjusts the channel store according to current cluster states.
|
||||
func (c *ChannelManagerImpl) Startup(ctx context.Context, legacyNodes, allNodes []int64) error {
|
||||
c.ctx = ctx
|
||||
channels := c.store.GetNodesChannels()
|
||||
// Retrieve the current old nodes.
|
||||
oNodes := make([]int64, 0, len(channels))
|
||||
for _, c := range channels {
|
||||
oNodes = append(oNodes, c.NodeID)
|
||||
}
|
||||
|
||||
// Process watch states for old nodes.
|
||||
oldOnLines := c.getOldOnlines(allNodes, oNodes)
|
||||
if err := c.checkOldNodes(oldOnLines); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Add new online nodes to the cluster.
|
||||
newOnLines := c.getNewOnLines(allNodes, oNodes)
|
||||
for _, n := range newOnLines {
|
||||
if err := c.AddNode(n); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// Remove new offline nodes from the cluster.
|
||||
offLines := c.getOffLines(allNodes, oNodes)
|
||||
for _, n := range offLines {
|
||||
if err := c.DeleteNode(n); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
// Unwatch and drop channel with drop flag.
|
||||
c.unwatchDroppedChannels()
|
||||
|
||||
checkerContext, cancel := context.WithCancel(ctx)
|
||||
c.stopChecker = cancel
|
||||
if c.stateChecker != nil {
|
||||
// TODO get revision from reload logic
|
||||
go c.stateChecker(checkerContext, common.LatestRevision)
|
||||
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",
|
||||
zap.Int64s("nodes", allNodes),
|
||||
zap.Int64s("oNodes", oNodes),
|
||||
zap.Int64s("old onlines", oldOnLines),
|
||||
zap.Int64s("new onlines", newOnLines),
|
||||
zap.Int64s("offLines", offLines))
|
||||
return nil
|
||||
}
|
||||
|
||||
// Close notifies the running checker.
|
||||
func (c *ChannelManagerImpl) Close() {
|
||||
if c.stopChecker != nil {
|
||||
c.stopChecker()
|
||||
}
|
||||
}
|
||||
|
||||
// checkOldNodes processes the existing watch channels when starting up.
|
||||
// ToWatch get startTs and timeoutTs, start timer
|
||||
// WatchSuccess ignore
|
||||
// WatchFail ToRelease
|
||||
// ToRelase get startTs and timeoutTs, start timer
|
||||
// ReleaseSuccess remove
|
||||
// ReleaseFail clean up and remove
|
||||
func (c *ChannelManagerImpl) checkOldNodes(nodes []UniqueID) error {
|
||||
// Load all the watch infos before processing
|
||||
nodeWatchInfos := make(map[UniqueID][]*datapb.ChannelWatchInfo)
|
||||
for _, nodeID := range nodes {
|
||||
watchInfos, err := c.stateTimer.loadAllChannels(nodeID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
nodeWatchInfos[nodeID] = watchInfos
|
||||
}
|
||||
|
||||
for nodeID, watchInfos := range nodeWatchInfos {
|
||||
for _, info := range watchInfos {
|
||||
channelName := info.GetVchan().GetChannelName()
|
||||
checkInterval := Params.DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second)
|
||||
|
||||
log.Info("processing watch info",
|
||||
zap.String("watch state", info.GetState().String()),
|
||||
zap.String("channelName", channelName))
|
||||
|
||||
switch info.GetState() {
|
||||
case datapb.ChannelWatchState_ToWatch, datapb.ChannelWatchState_Uncomplete:
|
||||
c.stateTimer.startOne(datapb.ChannelWatchState_ToWatch, channelName, nodeID, checkInterval)
|
||||
|
||||
case datapb.ChannelWatchState_WatchFailure:
|
||||
if err := c.Release(nodeID, channelName); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
case datapb.ChannelWatchState_ToRelease:
|
||||
c.stateTimer.startOne(datapb.ChannelWatchState_ToRelease, channelName, nodeID, checkInterval)
|
||||
|
||||
case datapb.ChannelWatchState_ReleaseSuccess:
|
||||
if err := c.Reassign(nodeID, channelName); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
case datapb.ChannelWatchState_ReleaseFailure:
|
||||
if err := c.CleanupAndReassign(nodeID, channelName); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// unwatchDroppedChannels removes drops channel that are marked to drop.
|
||||
func (c *ChannelManagerImpl) unwatchDroppedChannels() {
|
||||
nodeChannels := c.store.GetNodesChannels()
|
||||
for _, nodeChannel := range nodeChannels {
|
||||
for _, ch := range nodeChannel.Channels {
|
||||
if !c.isMarkedDrop(ch.GetName()) {
|
||||
continue
|
||||
}
|
||||
err := c.remove(nodeChannel.NodeID, ch)
|
||||
if err != nil {
|
||||
log.Warn("unable to remove channel", zap.String("channel", ch.GetName()), zap.Error(err))
|
||||
continue
|
||||
}
|
||||
err = c.h.FinishDropChannel(ch.GetName(), ch.GetCollectionID())
|
||||
if err != nil {
|
||||
log.Warn("FinishDropChannel failed when unwatchDroppedChannels", zap.String("channel", ch.GetName()), zap.Error(err))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *ChannelManagerImpl) bgCheckChannelsWork(ctx context.Context) {
|
||||
ticker := time.NewTicker(Params.DataCoordCfg.ChannelBalanceInterval.GetAsDuration(time.Second))
|
||||
defer ticker.Stop()
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
log.Info("background checking channels loop quit")
|
||||
return
|
||||
case <-ticker.C:
|
||||
if !Params.DataCoordCfg.AutoBalance.GetAsBool() {
|
||||
log.Info("auto balance disabled, skip auto bg check balance")
|
||||
continue
|
||||
}
|
||||
|
||||
c.mu.Lock()
|
||||
if !c.isSilent() {
|
||||
log.Info("ChannelManager is not silent, skip channel balance this round")
|
||||
} else {
|
||||
currCluster := c.store.GetNodesChannels()
|
||||
updates := c.balancePolicy(currCluster)
|
||||
if updates == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
log.Info("channel manager bg check balance", zap.Array("toReleases", updates))
|
||||
if err := c.updateWithTimer(updates, datapb.ChannelWatchState_ToRelease); err != nil {
|
||||
log.Warn("channel store update error", zap.Error(err))
|
||||
}
|
||||
}
|
||||
c.mu.Unlock()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// getOldOnlines returns a list of old online node ids in `old` and in `curr`.
|
||||
func (c *ChannelManagerImpl) getOldOnlines(curr []int64, old []int64) []int64 {
|
||||
mcurr := make(map[int64]struct{})
|
||||
ret := make([]int64, 0, len(old))
|
||||
for _, n := range curr {
|
||||
mcurr[n] = struct{}{}
|
||||
}
|
||||
for _, n := range old {
|
||||
if _, found := mcurr[n]; found {
|
||||
ret = append(ret, n)
|
||||
}
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
// getNewOnLines returns a list of new online node ids in `curr` but not in `old`.
|
||||
func (c *ChannelManagerImpl) getNewOnLines(curr []int64, old []int64) []int64 {
|
||||
mold := make(map[int64]struct{})
|
||||
ret := make([]int64, 0, len(curr))
|
||||
for _, n := range old {
|
||||
mold[n] = struct{}{}
|
||||
}
|
||||
for _, n := range curr {
|
||||
if _, found := mold[n]; !found {
|
||||
ret = append(ret, n)
|
||||
}
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
// getOffLines returns a list of new offline node ids in `old` but not in `curr`.
|
||||
func (c *ChannelManagerImpl) getOffLines(curr []int64, old []int64) []int64 {
|
||||
mcurr := make(map[int64]struct{})
|
||||
ret := make([]int64, 0, len(old))
|
||||
for _, n := range curr {
|
||||
mcurr[n] = struct{}{}
|
||||
}
|
||||
for _, n := range old {
|
||||
if _, found := mcurr[n]; !found {
|
||||
ret = append(ret, n)
|
||||
}
|
||||
}
|
||||
return ret
|
||||
}
|
||||
|
||||
// AddNode adds a new node to cluster and reassigns the node - channel mapping.
|
||||
func (c *ChannelManagerImpl) AddNode(nodeID int64) error {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
c.store.AddNode(nodeID)
|
||||
|
||||
bufferedUpdates, balanceUpdates := c.registerPolicy(c.store, nodeID)
|
||||
|
||||
updates := bufferedUpdates
|
||||
// try bufferedUpdates first
|
||||
if updates == nil {
|
||||
if !Params.DataCoordCfg.AutoBalance.GetAsBool() {
|
||||
log.Info("auto balance disabled, skip reassignment for balance", zap.Int64("registered node", nodeID))
|
||||
return nil
|
||||
}
|
||||
updates = balanceUpdates
|
||||
}
|
||||
|
||||
if updates == nil {
|
||||
log.Info("register node with no reassignment", zap.Int64("registered node", nodeID))
|
||||
return nil
|
||||
}
|
||||
|
||||
log.Info("register node", zap.Int64("registered node", nodeID), zap.Array("updates", updates))
|
||||
|
||||
state := datapb.ChannelWatchState_ToRelease
|
||||
|
||||
for _, u := range updates.Collect() {
|
||||
if u.Type == Delete && u.NodeID == bufferID {
|
||||
state = datapb.ChannelWatchState_ToWatch
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
return c.updateWithTimer(updates, state)
|
||||
}
|
||||
|
||||
// DeleteNode deletes the node from the cluster.
|
||||
// DeleteNode deletes the nodeID's watchInfos in Etcd and reassign the channels to other Nodes
|
||||
func (c *ChannelManagerImpl) DeleteNode(nodeID int64) error {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
nodeChannelInfo := c.store.GetNode(nodeID)
|
||||
if nodeChannelInfo == nil {
|
||||
c.store.RemoveNode(nodeID)
|
||||
return nil
|
||||
}
|
||||
|
||||
c.unsubAttempt(nodeChannelInfo)
|
||||
|
||||
updates := c.deregisterPolicy(c.store, nodeID)
|
||||
if updates == nil {
|
||||
c.store.RemoveNode(nodeID)
|
||||
return nil
|
||||
}
|
||||
log.Info("deregister node", zap.Int64("nodeID", nodeID), zap.Array("updates", updates))
|
||||
|
||||
var channels []RWChannel
|
||||
for _, op := range updates.Collect() {
|
||||
if op.Type == Delete {
|
||||
channels = op.Channels
|
||||
}
|
||||
}
|
||||
|
||||
chNames := make([]string, 0, len(channels))
|
||||
for _, ch := range channels {
|
||||
chNames = append(chNames, ch.GetName())
|
||||
}
|
||||
log.Info("remove timers for channel of the deregistered node",
|
||||
zap.Strings("channels", chNames), zap.Int64("nodeID", nodeID))
|
||||
c.stateTimer.removeTimers(chNames)
|
||||
|
||||
if err := c.updateWithTimer(updates, datapb.ChannelWatchState_ToWatch); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// No channels will be return
|
||||
c.store.RemoveNode(nodeID)
|
||||
return nil
|
||||
}
|
||||
|
||||
// unsubAttempt attempts to unsubscribe node-channel info from the channel.
|
||||
func (c *ChannelManagerImpl) unsubAttempt(ncInfo *NodeChannelInfo) {
|
||||
if ncInfo == nil {
|
||||
return
|
||||
}
|
||||
|
||||
if c.msgstreamFactory == nil {
|
||||
log.Warn("msgstream factory is not set")
|
||||
return
|
||||
}
|
||||
|
||||
nodeID := ncInfo.NodeID
|
||||
for _, ch := range ncInfo.Channels {
|
||||
// align to datanode subname, using vchannel name
|
||||
subName := fmt.Sprintf("%s-%d-%s", Params.CommonCfg.DataNodeSubName.GetValue(), nodeID, ch.GetName())
|
||||
pchannelName := funcutil.ToPhysicalChannel(ch.GetName())
|
||||
msgstream.UnsubscribeChannels(c.ctx, c.msgstreamFactory, subName, []string{pchannelName})
|
||||
}
|
||||
}
|
||||
|
||||
// Watch tries to add the channel to cluster. Watch is a no op if the channel already exists.
|
||||
func (c *ChannelManagerImpl) Watch(ctx context.Context, ch RWChannel) error {
|
||||
log := log.Ctx(ctx)
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
updates := c.assignPolicy(c.store, []RWChannel{ch})
|
||||
if updates == nil {
|
||||
return nil
|
||||
}
|
||||
log.Info("try to update channel watch info with ToWatch state",
|
||||
zap.String("channel", ch.String()),
|
||||
zap.Array("updates", updates))
|
||||
|
||||
err := c.updateWithTimer(updates, datapb.ChannelWatchState_ToWatch)
|
||||
if err != nil {
|
||||
log.Warn("fail to update channel watch info with ToWatch state",
|
||||
zap.String("channel", ch.String()), zap.Array("updates", updates), zap.Error(err))
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// fillChannelWatchInfoWithState updates the channel op by filling in channel watch info.
|
||||
func (c *ChannelManagerImpl) fillChannelWatchInfoWithState(op *ChannelOp, state datapb.ChannelWatchState) []string {
|
||||
channelsWithTimer := []string{}
|
||||
startTs := time.Now().Unix()
|
||||
checkInterval := Params.DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second)
|
||||
for _, ch := range op.Channels {
|
||||
vcInfo := c.h.GetDataVChanPositions(ch, allPartitionID)
|
||||
info := &datapb.ChannelWatchInfo{
|
||||
Vchan: vcInfo,
|
||||
StartTs: startTs,
|
||||
State: state,
|
||||
Schema: ch.GetSchema(),
|
||||
}
|
||||
|
||||
// Only set timer for watchInfo not from bufferID
|
||||
if op.NodeID != bufferID {
|
||||
c.stateTimer.startOne(state, ch.GetName(), op.NodeID, checkInterval)
|
||||
channelsWithTimer = append(channelsWithTimer, ch.GetName())
|
||||
}
|
||||
|
||||
ch.UpdateWatchInfo(info)
|
||||
}
|
||||
return channelsWithTimer
|
||||
}
|
||||
|
||||
// GetAssignedChannels gets channels info of registered nodes.
|
||||
func (c *ChannelManagerImpl) GetAssignedChannels() []*NodeChannelInfo {
|
||||
c.mu.RLock()
|
||||
defer c.mu.RUnlock()
|
||||
|
||||
return c.store.GetNodesChannels()
|
||||
}
|
||||
|
||||
// GetBufferChannels gets buffer channels.
|
||||
func (c *ChannelManagerImpl) GetBufferChannels() *NodeChannelInfo {
|
||||
c.mu.RLock()
|
||||
defer c.mu.RUnlock()
|
||||
|
||||
return c.store.GetBufferChannelInfo()
|
||||
}
|
||||
|
||||
// GetNodeChannelsByCollectionID gets all node channels map of the collection
|
||||
func (c *ChannelManagerImpl) GetNodeChannelsByCollectionID(collectionID UniqueID) map[UniqueID][]string {
|
||||
c.mu.RLock()
|
||||
defer c.mu.RUnlock()
|
||||
return c.store.GetNodeChannelsByCollectionID(collectionID)
|
||||
}
|
||||
|
||||
// Get all channels belong to the collection
|
||||
func (c *ChannelManagerImpl) GetChannelsByCollectionID(collectionID UniqueID) []RWChannel {
|
||||
channels := make([]RWChannel, 0)
|
||||
for _, nodeChannels := range c.GetAssignedChannels() {
|
||||
for _, ch := range nodeChannels.Channels {
|
||||
if ch.GetCollectionID() == collectionID {
|
||||
channels = append(channels, ch)
|
||||
}
|
||||
}
|
||||
}
|
||||
return channels
|
||||
}
|
||||
|
||||
// Get all channel names belong to the collection
|
||||
func (c *ChannelManagerImpl) GetChannelNamesByCollectionID(collectionID UniqueID) []string {
|
||||
channels := c.GetChannelsByCollectionID(collectionID)
|
||||
return lo.Map(channels, func(channel RWChannel, _ int) string {
|
||||
return channel.GetName()
|
||||
})
|
||||
}
|
||||
|
||||
// Match checks and returns whether the node ID and channel match.
|
||||
// use vchannel
|
||||
func (c *ChannelManagerImpl) Match(nodeID int64, channel string) bool {
|
||||
c.mu.RLock()
|
||||
defer c.mu.RUnlock()
|
||||
|
||||
info := c.store.GetNode(nodeID)
|
||||
if info == nil {
|
||||
return false
|
||||
}
|
||||
|
||||
if _, ok := info.Channels[channel]; ok {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
// FindWatcher finds the datanode watching the provided channel.
|
||||
func (c *ChannelManagerImpl) FindWatcher(channelName string) (int64, error) {
|
||||
c.mu.RLock()
|
||||
defer c.mu.RUnlock()
|
||||
|
||||
infos := c.store.GetNodesChannels()
|
||||
for _, info := range infos {
|
||||
if _, ok := info.Channels[channelName]; ok {
|
||||
return info.NodeID, nil
|
||||
}
|
||||
}
|
||||
|
||||
// channel in buffer
|
||||
bufferInfo := c.store.GetBufferChannelInfo()
|
||||
if _, ok := bufferInfo.Channels[channelName]; ok {
|
||||
return bufferID, errChannelInBuffer
|
||||
}
|
||||
return 0, errChannelNotWatched
|
||||
}
|
||||
|
||||
// RemoveChannel removes the channel from channel manager.
|
||||
func (c *ChannelManagerImpl) RemoveChannel(channelName string) error {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
nodeID, ch := c.findChannel(channelName)
|
||||
if ch == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
return c.remove(nodeID, ch)
|
||||
}
|
||||
|
||||
// remove deletes the nodeID-channel pair from data store.
|
||||
func (c *ChannelManagerImpl) remove(nodeID int64, ch RWChannel) error {
|
||||
op := NewChannelOpSet(NewDeleteOp(nodeID, ch))
|
||||
log.Info("remove channel assignment",
|
||||
zap.Int64("nodeID to be removed", nodeID),
|
||||
zap.String("channel", ch.GetName()),
|
||||
zap.Int64("collectionID", ch.GetCollectionID()))
|
||||
if err := c.store.Update(op); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *ChannelManagerImpl) findChannel(channelName string) (int64, RWChannel) {
|
||||
infos := c.store.GetNodesChannels()
|
||||
for _, info := range infos {
|
||||
if channelInfo, ok := info.Channels[channelName]; ok {
|
||||
return info.NodeID, channelInfo
|
||||
}
|
||||
}
|
||||
return 0, nil
|
||||
}
|
||||
|
||||
type ackType = int
|
||||
|
||||
const (
|
||||
invalidAck = iota
|
||||
watchSuccessAck
|
||||
watchFailAck
|
||||
watchTimeoutAck
|
||||
releaseSuccessAck
|
||||
releaseFailAck
|
||||
releaseTimeoutAck
|
||||
)
|
||||
|
||||
type ackEvent struct {
|
||||
ackType ackType
|
||||
channelName string
|
||||
nodeID UniqueID
|
||||
}
|
||||
|
||||
func (c *ChannelManagerImpl) updateWithTimer(updates *ChannelOpSet, state datapb.ChannelWatchState) error {
|
||||
channelsWithTimer := []string{}
|
||||
for _, op := range updates.Collect() {
|
||||
if op.Type != Delete {
|
||||
channelsWithTimer = append(channelsWithTimer, c.fillChannelWatchInfoWithState(op, state)...)
|
||||
}
|
||||
}
|
||||
|
||||
err := c.store.Update(updates)
|
||||
if err != nil {
|
||||
log.Warn("fail to update", zap.Array("updates", updates), zap.Error(err))
|
||||
c.stateTimer.removeTimers(channelsWithTimer)
|
||||
}
|
||||
c.lastActiveTimestamp = time.Now()
|
||||
return err
|
||||
}
|
||||
|
||||
func (c *ChannelManagerImpl) processAck(e *ackEvent) {
|
||||
c.stateTimer.stopIfExist(e)
|
||||
|
||||
switch e.ackType {
|
||||
case invalidAck:
|
||||
log.Warn("detected invalid Ack", zap.String("channelName", e.channelName))
|
||||
|
||||
case watchSuccessAck:
|
||||
log.Info("datanode successfully watched channel", zap.Int64("nodeID", e.nodeID), zap.String("channelName", e.channelName))
|
||||
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)
|
||||
if err != nil {
|
||||
log.Warn("fail to set channels to release for watch failure ACKs",
|
||||
zap.Int64("nodeID", e.nodeID), zap.String("channelName", e.channelName), zap.Error(err))
|
||||
}
|
||||
case releaseFailAck, releaseTimeoutAck: // failure acks from toRelease
|
||||
// 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)
|
||||
if err != nil {
|
||||
log.Warn("fail to clean and reassign channels for release failure ACKs",
|
||||
zap.Int64("nodeID", e.nodeID), zap.String("channelName", e.channelName), zap.Error(err))
|
||||
}
|
||||
|
||||
case releaseSuccessAck:
|
||||
// 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)
|
||||
if err != nil {
|
||||
log.Warn("fail to response to release success ACK",
|
||||
zap.Int64("nodeID", e.nodeID), zap.String("channelName", e.channelName), zap.Error(err))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
type channelStateChecker func(context.Context, int64)
|
||||
|
||||
func (c *ChannelManagerImpl) watchChannelStatesLoop(ctx context.Context, revision int64) {
|
||||
defer logutil.LogPanic()
|
||||
|
||||
// REF MEP#7 watchInfo paths are orgnized as: [prefix]/channel/{node_id}/{channel_name}
|
||||
watchPrefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue()
|
||||
// TODO, this is risky, we'd better watch etcd with revision rather simply a path
|
||||
var etcdWatcher clientv3.WatchChan
|
||||
var timeoutWatcher chan *ackEvent
|
||||
if revision == common.LatestRevision {
|
||||
etcdWatcher, timeoutWatcher = c.stateTimer.getWatchers(watchPrefix)
|
||||
} else {
|
||||
etcdWatcher, timeoutWatcher = c.stateTimer.getWatchersWithRevision(watchPrefix, revision)
|
||||
}
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
log.Info("watch etcd loop quit")
|
||||
return
|
||||
case ackEvent := <-timeoutWatcher:
|
||||
log.Info("receive timeout acks from state watcher",
|
||||
zap.Int("state", ackEvent.ackType),
|
||||
zap.Int64("nodeID", ackEvent.nodeID), zap.String("channelName", ackEvent.channelName))
|
||||
c.processAck(ackEvent)
|
||||
case event, ok := <-etcdWatcher:
|
||||
if !ok {
|
||||
log.Warn("datacoord failed to watch channel, return")
|
||||
// rewatch for transient network error, session handles process quiting if connect is not recoverable
|
||||
go c.watchChannelStatesLoop(ctx, revision)
|
||||
return
|
||||
}
|
||||
|
||||
if err := event.Err(); err != nil {
|
||||
log.Warn("datacoord watch channel hit error", zap.Error(event.Err()))
|
||||
// https://github.com/etcd-io/etcd/issues/8980
|
||||
// TODO add list and wathc with revision
|
||||
if event.Err() == v3rpc.ErrCompacted {
|
||||
go c.watchChannelStatesLoop(ctx, event.CompactRevision)
|
||||
return
|
||||
}
|
||||
// if watch loop return due to event canceled, the datacoord is not functional anymore
|
||||
log.Panic("datacoord is not functional for event canceled", zap.Error(err))
|
||||
return
|
||||
}
|
||||
|
||||
revision = event.Header.GetRevision() + 1
|
||||
for _, evt := range event.Events {
|
||||
if evt.Type == clientv3.EventTypeDelete {
|
||||
continue
|
||||
}
|
||||
key := string(evt.Kv.Key)
|
||||
watchInfo, err := parseWatchInfo(key, evt.Kv.Value)
|
||||
if err != nil {
|
||||
log.Warn("fail to parse watch info", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
// runnging states
|
||||
state := watchInfo.GetState()
|
||||
if state == datapb.ChannelWatchState_ToWatch ||
|
||||
state == datapb.ChannelWatchState_ToRelease ||
|
||||
state == datapb.ChannelWatchState_Uncomplete {
|
||||
c.stateTimer.resetIfExist(watchInfo.GetVchan().ChannelName, Params.DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second))
|
||||
log.Info("tickle update, timer delay", zap.String("channel", watchInfo.GetVchan().ChannelName), zap.Int32("progress", watchInfo.Progress))
|
||||
continue
|
||||
}
|
||||
|
||||
nodeID, err := parseNodeKey(key)
|
||||
if err != nil {
|
||||
log.Warn("fail to parse node from key", zap.String("key", key), zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
ackEvent := parseAckEvent(nodeID, watchInfo)
|
||||
c.processAck(ackEvent)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Release writes ToRelease channel watch states for a channel
|
||||
func (c *ChannelManagerImpl) Release(nodeID UniqueID, channelName string) error {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
|
||||
toReleaseChannel := c.getChannelByNodeAndName(nodeID, channelName)
|
||||
if toReleaseChannel == nil {
|
||||
return fmt.Errorf("fail to find matching nodeID: %d with channelName: %s", nodeID, channelName)
|
||||
}
|
||||
|
||||
toReleaseUpdates := NewChannelOpSet(NewAddOp(nodeID, toReleaseChannel))
|
||||
err := c.updateWithTimer(toReleaseUpdates, datapb.ChannelWatchState_ToRelease)
|
||||
if err != nil {
|
||||
log.Warn("fail to update to release with timer", zap.Array("to release updates", toReleaseUpdates))
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
// Reassign reassigns a channel to another DataNode.
|
||||
func (c *ChannelManagerImpl) Reassign(originNodeID UniqueID, channelName string) error {
|
||||
c.mu.RLock()
|
||||
ch := c.getChannelByNodeAndName(originNodeID, channelName)
|
||||
if ch == nil {
|
||||
c.mu.RUnlock()
|
||||
return fmt.Errorf("fail to find matching nodeID: %d with channelName: %s", originNodeID, channelName)
|
||||
}
|
||||
c.mu.RUnlock()
|
||||
|
||||
reallocates := NewNodeChannelInfo(originNodeID, ch)
|
||||
isDropped := c.isMarkedDrop(channelName)
|
||||
|
||||
if isDropped {
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
if err := c.remove(originNodeID, ch); err != nil {
|
||||
return fmt.Errorf("failed to remove watch info: %v,%s", ch, err.Error())
|
||||
}
|
||||
if err := c.h.FinishDropChannel(channelName, ch.GetCollectionID()); err != nil {
|
||||
return fmt.Errorf("FinishDropChannel failed, err=%w", err)
|
||||
}
|
||||
log.Info("removed channel assignment", zap.String("channelName", channelName))
|
||||
return nil
|
||||
}
|
||||
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
// Reassign policy won't choose the original node when a reassigning a channel.
|
||||
updates := c.reassignPolicy(c.store, []*NodeChannelInfo{reallocates})
|
||||
if updates == nil {
|
||||
// Skip the remove if reassign to the original node.
|
||||
log.Warn("failed to reassign channel to other nodes, assigning to the original DataNode",
|
||||
zap.Int64("nodeID", originNodeID),
|
||||
zap.String("channelName", channelName))
|
||||
updates = NewChannelOpSet(NewAddOp(originNodeID, ch))
|
||||
}
|
||||
|
||||
log.Info("channel manager reassigning channels",
|
||||
zap.Int64("old node ID", originNodeID),
|
||||
zap.Array("updates", updates))
|
||||
return c.updateWithTimer(updates, datapb.ChannelWatchState_ToWatch)
|
||||
}
|
||||
|
||||
// CleanupAndReassign tries to clean up datanode's subscription, and then reassigns the channel to another DataNode.
|
||||
func (c *ChannelManagerImpl) CleanupAndReassign(nodeID UniqueID, channelName string) error {
|
||||
c.mu.RLock()
|
||||
chToCleanUp := c.getChannelByNodeAndName(nodeID, channelName)
|
||||
if chToCleanUp == nil {
|
||||
c.mu.RUnlock()
|
||||
return fmt.Errorf("failed to find matching channel: %s and node: %d", channelName, nodeID)
|
||||
}
|
||||
c.mu.RUnlock()
|
||||
|
||||
if c.msgstreamFactory == nil {
|
||||
log.Warn("msgstream factory is not set, unable to clean up topics")
|
||||
} else {
|
||||
subName := fmt.Sprintf("%s-%d-%d", Params.CommonCfg.DataNodeSubName.GetValue(), nodeID, chToCleanUp.GetCollectionID())
|
||||
pchannelName := funcutil.ToPhysicalChannel(channelName)
|
||||
msgstream.UnsubscribeChannels(c.ctx, c.msgstreamFactory, subName, []string{pchannelName})
|
||||
}
|
||||
|
||||
reallocates := NewNodeChannelInfo(nodeID, chToCleanUp)
|
||||
isDropped := c.isMarkedDrop(channelName)
|
||||
|
||||
c.mu.Lock()
|
||||
defer c.mu.Unlock()
|
||||
if isDropped {
|
||||
if err := c.remove(nodeID, chToCleanUp); err != nil {
|
||||
return fmt.Errorf("failed to remove watch info: %v,%s", chToCleanUp, err.Error())
|
||||
}
|
||||
|
||||
log.Info("try to cleanup removal flag ", zap.String("channelName", channelName))
|
||||
if err := c.h.FinishDropChannel(channelName, chToCleanUp.GetCollectionID()); err != nil {
|
||||
return fmt.Errorf("FinishDropChannel failed, err=%w", err)
|
||||
}
|
||||
|
||||
log.Info("removed channel assignment", zap.Any("channel", chToCleanUp))
|
||||
return nil
|
||||
}
|
||||
|
||||
// Reassign policy won't choose the original node when a reassigning a channel.
|
||||
updates := c.reassignPolicy(c.store, []*NodeChannelInfo{reallocates})
|
||||
if updates == nil {
|
||||
// Skip the remove if reassign to the original node.
|
||||
log.Warn("failed to reassign channel to other nodes, add channel to the original node",
|
||||
zap.Int64("node ID", nodeID),
|
||||
zap.String("channelName", channelName))
|
||||
updates = NewChannelOpSet(NewAddOp(nodeID, chToCleanUp))
|
||||
}
|
||||
|
||||
log.Info("channel manager reassigning channels",
|
||||
zap.Int64("old nodeID", nodeID),
|
||||
zap.Array("updates", updates))
|
||||
return c.updateWithTimer(updates, datapb.ChannelWatchState_ToWatch)
|
||||
}
|
||||
|
||||
func (c *ChannelManagerImpl) getChannelByNodeAndName(nodeID UniqueID, channelName string) RWChannel {
|
||||
if nodeChannelInfo := c.store.GetNode(nodeID); nodeChannelInfo != nil {
|
||||
if ch, ok := nodeChannelInfo.Channels[channelName]; ok {
|
||||
return ch
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *ChannelManagerImpl) GetCollectionIDByChannel(channelName string) (bool, UniqueID) {
|
||||
for _, nodeChannel := range c.GetAssignedChannels() {
|
||||
if ch, ok := nodeChannel.Channels[channelName]; ok {
|
||||
return true, ch.GetCollectionID()
|
||||
}
|
||||
}
|
||||
return false, 0
|
||||
}
|
||||
|
||||
func (c *ChannelManagerImpl) GetChannel(nodeID int64, channelName string) (RWChannel, bool) {
|
||||
c.mu.RLock()
|
||||
defer c.mu.RUnlock()
|
||||
|
||||
ch := c.getChannelByNodeAndName(nodeID, channelName)
|
||||
return ch, ch != nil
|
||||
}
|
||||
|
||||
func (c *ChannelManagerImpl) isMarkedDrop(channel string) bool {
|
||||
return c.h.CheckShouldDropChannel(channel)
|
||||
}
|
||||
|
||||
func (c *ChannelManagerImpl) isSilent() bool {
|
||||
if c.stateTimer.hasRunningTimers() {
|
||||
return false
|
||||
}
|
||||
return time.Since(c.lastActiveTimestamp) >= Params.DataCoordCfg.ChannelBalanceSilentDuration.GetAsDuration(time.Second)
|
||||
}
|
File diff suppressed because it is too large
Load Diff
@ -32,7 +32,6 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/kv"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
@ -154,9 +153,7 @@ func (op *ChannelOp) BuildKV() (map[string]string, []string, error) {
|
||||
switch op.Type {
|
||||
case Add, Watch, Release:
|
||||
tmpWatchInfo := proto.Clone(ch.GetWatchInfo()).(*datapb.ChannelWatchInfo)
|
||||
if paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.GetAsBool() {
|
||||
tmpWatchInfo.Vchan = reduceVChanSize(tmpWatchInfo.GetVchan())
|
||||
}
|
||||
tmpWatchInfo.Vchan = reduceVChanSize(tmpWatchInfo.GetVchan())
|
||||
info, err := proto.Marshal(tmpWatchInfo)
|
||||
if err != nil {
|
||||
return saves, removals, err
|
||||
|
@ -22,91 +22,15 @@ import (
|
||||
"testing"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/suite"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/kv/mocks"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/kv"
|
||||
"github.com/milvus-io/milvus/pkg/kv/predicates"
|
||||
"github.com/milvus-io/milvus/pkg/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/testutils"
|
||||
)
|
||||
|
||||
func genNodeChannelInfos(id int64, num int) *NodeChannelInfo {
|
||||
channels := make([]RWChannel, 0, num)
|
||||
for i := 0; i < num; i++ {
|
||||
name := fmt.Sprintf("ch%d", i)
|
||||
channels = append(channels, &channelMeta{Name: name, CollectionID: 1, WatchInfo: &datapb.ChannelWatchInfo{}})
|
||||
}
|
||||
return NewNodeChannelInfo(id, channels...)
|
||||
}
|
||||
|
||||
func genChannelOperationsV1(from, to int64, num int) *ChannelOpSet {
|
||||
channels := make([]RWChannel, 0, num)
|
||||
for i := 0; i < num; i++ {
|
||||
name := fmt.Sprintf("ch%d", i)
|
||||
channels = append(channels, &channelMeta{Name: name, CollectionID: 1, WatchInfo: &datapb.ChannelWatchInfo{}})
|
||||
}
|
||||
|
||||
ops := NewChannelOpSet(
|
||||
NewAddOp(to, channels...),
|
||||
NewDeleteOp(from, channels...),
|
||||
)
|
||||
return ops
|
||||
}
|
||||
|
||||
func TestChannelStore_Update(t *testing.T) {
|
||||
enableRPCK := paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.Key
|
||||
paramtable.Get().Save(enableRPCK, "false")
|
||||
defer paramtable.Get().Reset(enableRPCK)
|
||||
txnKv := mocks.NewTxnKV(t)
|
||||
txnKv.EXPECT().MultiSaveAndRemove(mock.Anything, mock.Anything).Run(func(saves map[string]string, removals []string, preds ...predicates.Predicate) {
|
||||
assert.False(t, len(saves)+len(removals) > 64, "too many operations")
|
||||
}).Return(nil)
|
||||
|
||||
type fields struct {
|
||||
store kv.TxnKV
|
||||
channelsInfo map[int64]*NodeChannelInfo
|
||||
}
|
||||
type args struct {
|
||||
opSet *ChannelOpSet
|
||||
}
|
||||
tests := []struct {
|
||||
name string
|
||||
fields fields
|
||||
args args
|
||||
wantErr bool
|
||||
}{
|
||||
{
|
||||
"test more than 128 operations",
|
||||
fields{
|
||||
txnKv,
|
||||
map[int64]*NodeChannelInfo{
|
||||
1: genNodeChannelInfos(1, 500),
|
||||
2: NewNodeChannelInfo(2),
|
||||
},
|
||||
},
|
||||
args{
|
||||
genChannelOperationsV1(1, 2, 250),
|
||||
},
|
||||
false,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
c := &ChannelStore{
|
||||
store: tt.fields.store,
|
||||
channelsInfo: tt.fields.channelsInfo,
|
||||
}
|
||||
err := c.Update(tt.args.opSet)
|
||||
assert.Equal(t, tt.wantErr, err != nil)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
type ChannelStoreReloadSuite struct {
|
||||
testutils.PromMetricsSuite
|
||||
|
||||
|
@ -487,16 +487,9 @@ func (s *Server) initCluster() error {
|
||||
s.sessionManager = NewSessionManagerImpl(withSessionCreator(s.dataNodeCreator))
|
||||
|
||||
var err error
|
||||
if paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.GetAsBool() {
|
||||
s.channelManager, err = NewChannelManagerV2(s.watchClient, s.handler, s.sessionManager, s.allocator, withCheckerV2())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
} else {
|
||||
s.channelManager, err = NewChannelManager(s.watchClient, s.handler, withMsgstreamFactory(s.factory), withStateChecker(), withBgChecker())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
s.channelManager, err = NewChannelManagerV2(s.watchClient, s.handler, s.sessionManager, s.allocator, withCheckerV2())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
s.cluster = NewClusterImpl(s.sessionManager, s.channelManager)
|
||||
return nil
|
||||
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package channel
|
||||
|
||||
import (
|
||||
"context"
|
||||
@ -22,19 +22,21 @@ import (
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"go.uber.org/atomic"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/datanode/pipeline"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/lifetime"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
type (
|
||||
releaseFunc func(channel string)
|
||||
watchFunc func(ctx context.Context, dn *DataNode, info *datapb.ChannelWatchInfo, tickler *tickler) (*dataSyncService, error)
|
||||
watchFunc func(ctx context.Context, pipelineParams *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler) (*pipeline.DataSyncService, error)
|
||||
)
|
||||
|
||||
type ChannelManager interface {
|
||||
@ -45,10 +47,10 @@ type ChannelManager interface {
|
||||
}
|
||||
|
||||
type ChannelManagerImpl struct {
|
||||
mu sync.RWMutex
|
||||
dn *DataNode
|
||||
mu sync.RWMutex
|
||||
pipelineParams *util.PipelineParams
|
||||
|
||||
fgManager FlowgraphManager
|
||||
fgManager pipeline.FlowgraphManager
|
||||
|
||||
communicateCh chan *opState
|
||||
opRunners *typeutil.ConcurrentMap[string, *opRunner] // channel -> runner
|
||||
@ -60,16 +62,16 @@ type ChannelManagerImpl struct {
|
||||
closeWaiter sync.WaitGroup
|
||||
}
|
||||
|
||||
func NewChannelManager(dn *DataNode) *ChannelManagerImpl {
|
||||
func NewChannelManager(pipelineParams *util.PipelineParams, fgManager pipeline.FlowgraphManager) *ChannelManagerImpl {
|
||||
cm := ChannelManagerImpl{
|
||||
dn: dn,
|
||||
fgManager: dn.flowgraphManager,
|
||||
pipelineParams: pipelineParams,
|
||||
fgManager: fgManager,
|
||||
|
||||
communicateCh: make(chan *opState, 100),
|
||||
opRunners: typeutil.NewConcurrentMap[string, *opRunner](),
|
||||
abnormals: typeutil.NewConcurrentMap[int64, string](),
|
||||
|
||||
releaseFunc: dn.flowgraphManager.RemoveFlowgraph,
|
||||
releaseFunc: fgManager.RemoveFlowgraph,
|
||||
|
||||
closeCh: lifetime.NewSafeChan(),
|
||||
}
|
||||
@ -209,7 +211,7 @@ func (m *ChannelManagerImpl) handleOpState(opState *opState) {
|
||||
}
|
||||
|
||||
func (m *ChannelManagerImpl) getOrCreateRunner(channel string) *opRunner {
|
||||
runner, loaded := m.opRunners.GetOrInsert(channel, NewOpRunner(channel, m.dn, m.releaseFunc, executeWatch, m.communicateCh))
|
||||
runner, loaded := m.opRunners.GetOrInsert(channel, NewOpRunner(channel, m.pipelineParams, m.releaseFunc, executeWatch, m.communicateCh))
|
||||
if !loaded {
|
||||
runner.Start()
|
||||
}
|
||||
@ -224,17 +226,17 @@ func (m *ChannelManagerImpl) finishOp(opID int64, channel string) {
|
||||
}
|
||||
|
||||
type opInfo struct {
|
||||
tickler *tickler
|
||||
tickler *util.Tickler
|
||||
}
|
||||
|
||||
type opRunner struct {
|
||||
channel string
|
||||
dn *DataNode
|
||||
releaseFunc releaseFunc
|
||||
watchFunc watchFunc
|
||||
channel string
|
||||
pipelineParams *util.PipelineParams
|
||||
releaseFunc releaseFunc
|
||||
watchFunc watchFunc
|
||||
|
||||
guard sync.RWMutex
|
||||
allOps map[UniqueID]*opInfo // opID -> tickler
|
||||
allOps map[util.UniqueID]*opInfo // opID -> tickler
|
||||
opsInQueue chan *datapb.ChannelWatchInfo
|
||||
resultCh chan *opState
|
||||
|
||||
@ -242,16 +244,16 @@ type opRunner struct {
|
||||
closeWg sync.WaitGroup
|
||||
}
|
||||
|
||||
func NewOpRunner(channel string, dn *DataNode, releaseF releaseFunc, watchF watchFunc, resultCh chan *opState) *opRunner {
|
||||
func NewOpRunner(channel string, pipelineParams *util.PipelineParams, releaseF releaseFunc, watchF watchFunc, resultCh chan *opState) *opRunner {
|
||||
return &opRunner{
|
||||
channel: channel,
|
||||
dn: dn,
|
||||
releaseFunc: releaseF,
|
||||
watchFunc: watchF,
|
||||
opsInQueue: make(chan *datapb.ChannelWatchInfo, 10),
|
||||
allOps: make(map[UniqueID]*opInfo),
|
||||
resultCh: resultCh,
|
||||
closeCh: lifetime.NewSafeChan(),
|
||||
channel: channel,
|
||||
pipelineParams: pipelineParams,
|
||||
releaseFunc: releaseF,
|
||||
watchFunc: watchF,
|
||||
opsInQueue: make(chan *datapb.ChannelWatchInfo, 10),
|
||||
allOps: make(map[util.UniqueID]*opInfo),
|
||||
resultCh: resultCh,
|
||||
closeCh: lifetime.NewSafeChan(),
|
||||
}
|
||||
}
|
||||
|
||||
@ -270,13 +272,13 @@ func (r *opRunner) Start() {
|
||||
}()
|
||||
}
|
||||
|
||||
func (r *opRunner) FinishOp(opID UniqueID) {
|
||||
func (r *opRunner) FinishOp(opID util.UniqueID) {
|
||||
r.guard.Lock()
|
||||
defer r.guard.Unlock()
|
||||
delete(r.allOps, opID)
|
||||
}
|
||||
|
||||
func (r *opRunner) Exist(opID UniqueID) bool {
|
||||
func (r *opRunner) Exist(opID util.UniqueID) bool {
|
||||
r.guard.RLock()
|
||||
defer r.guard.RUnlock()
|
||||
_, ok := r.allOps[opID]
|
||||
@ -334,7 +336,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState {
|
||||
opState.state = datapb.ChannelWatchState_WatchFailure
|
||||
return opState
|
||||
}
|
||||
tickler := newTickler()
|
||||
tickler := util.NewTickler()
|
||||
opInfo.tickler = tickler
|
||||
|
||||
var (
|
||||
@ -342,7 +344,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState {
|
||||
finishWaiter sync.WaitGroup
|
||||
)
|
||||
|
||||
watchTimeout := Params.DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second)
|
||||
watchTimeout := paramtable.Get().DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second)
|
||||
ctx, cancel := context.WithTimeout(context.Background(), watchTimeout)
|
||||
defer cancel()
|
||||
|
||||
@ -358,19 +360,19 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState {
|
||||
select {
|
||||
case <-timer.C:
|
||||
// watch timeout
|
||||
tickler.close()
|
||||
tickler.Close()
|
||||
cancel()
|
||||
log.Info("Stop timer for ToWatch operation timeout")
|
||||
return
|
||||
|
||||
case <-r.closeCh.CloseCh():
|
||||
// runner closed from outside
|
||||
tickler.close()
|
||||
tickler.Close()
|
||||
cancel()
|
||||
log.Info("Suspend ToWatch operation from outside of opRunner")
|
||||
return
|
||||
|
||||
case <-tickler.progressSig:
|
||||
case <-tickler.GetProgressSig():
|
||||
log.Info("Reset timer for tickler updated")
|
||||
timer.Reset(watchTimeout)
|
||||
|
||||
@ -387,7 +389,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState {
|
||||
|
||||
go func() {
|
||||
defer finishWaiter.Done()
|
||||
fg, err := r.watchFunc(ctx, r.dn, info, tickler)
|
||||
fg, err := r.watchFunc(ctx, r.pipelineParams, info, tickler)
|
||||
if err != nil {
|
||||
opState.state = datapb.ChannelWatchState_WatchFailure
|
||||
} else {
|
||||
@ -402,7 +404,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState {
|
||||
}
|
||||
|
||||
// releaseWithTimer will return ReleaseFailure after WatchTimeoutInterval
|
||||
func (r *opRunner) releaseWithTimer(releaseFunc releaseFunc, channel string, opID UniqueID) *opState {
|
||||
func (r *opRunner) releaseWithTimer(releaseFunc releaseFunc, channel string, opID util.UniqueID) *opState {
|
||||
opState := &opState{
|
||||
channel: channel,
|
||||
opID: opID,
|
||||
@ -416,7 +418,7 @@ func (r *opRunner) releaseWithTimer(releaseFunc releaseFunc, channel string, opI
|
||||
startTimer := func(finishWaiter *sync.WaitGroup) {
|
||||
defer finishWaiter.Done()
|
||||
|
||||
releaseTimeout := Params.DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second)
|
||||
releaseTimeout := paramtable.Get().DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second)
|
||||
timer := time.NewTimer(releaseTimeout)
|
||||
defer timer.Stop()
|
||||
|
||||
@ -474,61 +476,17 @@ type opState struct {
|
||||
channel string
|
||||
opID int64
|
||||
state datapb.ChannelWatchState
|
||||
fg *dataSyncService
|
||||
fg *pipeline.DataSyncService
|
||||
}
|
||||
|
||||
// executeWatch will always return, won't be stuck, either success or fail.
|
||||
func executeWatch(ctx context.Context, dn *DataNode, info *datapb.ChannelWatchInfo, tickler *tickler) (*dataSyncService, error) {
|
||||
dataSyncService, err := newDataSyncService(ctx, dn, info, tickler)
|
||||
func executeWatch(ctx context.Context, pipelineParams *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler) (*pipeline.DataSyncService, error) {
|
||||
dataSyncService, err := pipeline.NewDataSyncService(ctx, pipelineParams, info, tickler)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
dataSyncService.start()
|
||||
dataSyncService.Start()
|
||||
|
||||
return dataSyncService, nil
|
||||
}
|
||||
|
||||
// tickler counts every time when called inc(),
|
||||
type tickler struct {
|
||||
count *atomic.Int32
|
||||
total *atomic.Int32
|
||||
closedSig *atomic.Bool
|
||||
|
||||
progressSig chan struct{}
|
||||
}
|
||||
|
||||
func (t *tickler) inc() {
|
||||
t.count.Inc()
|
||||
t.progressSig <- struct{}{}
|
||||
}
|
||||
|
||||
func (t *tickler) setTotal(total int32) {
|
||||
t.total.Store(total)
|
||||
}
|
||||
|
||||
// progress returns the count over total if total is set
|
||||
// else just return the count number.
|
||||
func (t *tickler) progress() int32 {
|
||||
if t.total.Load() == 0 {
|
||||
return t.count.Load()
|
||||
}
|
||||
return (t.count.Load() / t.total.Load()) * 100
|
||||
}
|
||||
|
||||
func (t *tickler) close() {
|
||||
t.closedSig.CompareAndSwap(false, true)
|
||||
}
|
||||
|
||||
func (t *tickler) closed() bool {
|
||||
return t.closedSig.Load()
|
||||
}
|
||||
|
||||
func newTickler() *tickler {
|
||||
return &tickler{
|
||||
count: atomic.NewInt32(0),
|
||||
total: atomic.NewInt32(0),
|
||||
closedSig: atomic.NewBool(false),
|
||||
progressSig: make(chan struct{}, 200),
|
||||
}
|
||||
}
|
@ -14,24 +14,43 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package channel
|
||||
|
||||
import (
|
||||
"context"
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/suite"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/broker"
|
||||
"github.com/milvus-io/milvus/internal/datanode/pipeline"
|
||||
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/datanode/writebuffer"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
func TestMain(t *testing.M) {
|
||||
paramtable.Init()
|
||||
err := util.InitGlobalRateCollector()
|
||||
if err != nil {
|
||||
panic("init test failed, err = " + err.Error())
|
||||
}
|
||||
code := t.Run()
|
||||
os.Exit(code)
|
||||
}
|
||||
|
||||
func TestChannelManagerSuite(t *testing.T) {
|
||||
suite.Run(t, new(ChannelManagerSuite))
|
||||
}
|
||||
@ -41,11 +60,30 @@ func TestOpRunnerSuite(t *testing.T) {
|
||||
}
|
||||
|
||||
func (s *OpRunnerSuite) SetupTest() {
|
||||
ctx := context.Background()
|
||||
s.mockAlloc = allocator.NewMockAllocator(s.T())
|
||||
mockedBroker := broker.NewMockBroker(s.T())
|
||||
mockedBroker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).
|
||||
Return([]*datapb.SegmentInfo{}, nil).Maybe()
|
||||
|
||||
s.node = newIDLEDataNodeMock(ctx, schemapb.DataType_Int64)
|
||||
s.node.allocator = s.mockAlloc
|
||||
wbManager := writebuffer.NewMockBufferManager(s.T())
|
||||
wbManager.EXPECT().
|
||||
Register(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(nil).Maybe()
|
||||
|
||||
dispClient := msgdispatcher.NewMockClient(s.T())
|
||||
dispClient.EXPECT().Register(mock.Anything, mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(make(chan *msgstream.MsgPack), nil).Maybe()
|
||||
dispClient.EXPECT().Deregister(mock.Anything).Maybe()
|
||||
|
||||
s.pipelineParams = &util.PipelineParams{
|
||||
Ctx: context.TODO(),
|
||||
Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 0}},
|
||||
CheckpointUpdater: util.NewChannelCheckpointUpdater(mockedBroker),
|
||||
WriteBufferManager: wbManager,
|
||||
Broker: mockedBroker,
|
||||
DispClient: dispClient,
|
||||
SyncMgr: syncmgr.NewMockSyncManager(s.T()),
|
||||
Allocator: allocator.NewMockAllocator(s.T()),
|
||||
}
|
||||
}
|
||||
|
||||
func (s *OpRunnerSuite) TestWatchWithTimer() {
|
||||
@ -53,11 +91,12 @@ func (s *OpRunnerSuite) TestWatchWithTimer() {
|
||||
channel string = "ch-1"
|
||||
commuCh = make(chan *opState)
|
||||
)
|
||||
info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
|
||||
info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
|
||||
mockReleaseFunc := func(channel string) {
|
||||
log.Info("mock release func")
|
||||
}
|
||||
runner := NewOpRunner(channel, s.node, mockReleaseFunc, executeWatch, commuCh)
|
||||
|
||||
runner := NewOpRunner(channel, s.pipelineParams, mockReleaseFunc, executeWatch, commuCh)
|
||||
err := runner.Enqueue(info)
|
||||
s.Require().NoError(err)
|
||||
|
||||
@ -70,21 +109,21 @@ func (s *OpRunnerSuite) TestWatchWithTimer() {
|
||||
|
||||
func (s *OpRunnerSuite) TestWatchTimeout() {
|
||||
channel := "by-dev-rootcoord-dml-1000"
|
||||
paramtable.Get().Save(Params.DataCoordCfg.WatchTimeoutInterval.Key, "0.000001")
|
||||
defer paramtable.Get().Reset(Params.DataCoordCfg.WatchTimeoutInterval.Key)
|
||||
info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
|
||||
paramtable.Get().Save(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key, "0.000001")
|
||||
defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key)
|
||||
info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
|
||||
|
||||
sig := make(chan struct{})
|
||||
commuCh := make(chan *opState)
|
||||
|
||||
mockReleaseFunc := func(channel string) { log.Info("mock release func") }
|
||||
mockWatchFunc := func(ctx context.Context, dn *DataNode, info *datapb.ChannelWatchInfo, tickler *tickler) (*dataSyncService, error) {
|
||||
mockWatchFunc := func(ctx context.Context, param *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler) (*pipeline.DataSyncService, error) {
|
||||
<-ctx.Done()
|
||||
sig <- struct{}{}
|
||||
return nil, errors.New("timeout")
|
||||
}
|
||||
|
||||
runner := NewOpRunner(channel, s.node, mockReleaseFunc, mockWatchFunc, commuCh)
|
||||
runner := NewOpRunner(channel, s.pipelineParams, mockReleaseFunc, mockWatchFunc, commuCh)
|
||||
runner.Start()
|
||||
defer runner.Close()
|
||||
err := runner.Enqueue(info)
|
||||
@ -99,55 +138,40 @@ func (s *OpRunnerSuite) TestWatchTimeout() {
|
||||
|
||||
type OpRunnerSuite struct {
|
||||
suite.Suite
|
||||
node *DataNode
|
||||
mockAlloc *allocator.MockAllocator
|
||||
pipelineParams *util.PipelineParams
|
||||
}
|
||||
|
||||
type ChannelManagerSuite struct {
|
||||
suite.Suite
|
||||
|
||||
node *DataNode
|
||||
manager *ChannelManagerImpl
|
||||
pipelineParams *util.PipelineParams
|
||||
manager *ChannelManagerImpl
|
||||
}
|
||||
|
||||
func (s *ChannelManagerSuite) SetupTest() {
|
||||
ctx := context.Background()
|
||||
s.node = newIDLEDataNodeMock(ctx, schemapb.DataType_Int64)
|
||||
s.node.allocator = allocator.NewMockAllocator(s.T())
|
||||
s.node.flowgraphManager = newFlowgraphManager()
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
|
||||
s.manager = NewChannelManager(s.node)
|
||||
}
|
||||
wbManager := writebuffer.NewMockBufferManager(s.T())
|
||||
wbManager.EXPECT().
|
||||
Register(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).
|
||||
Return(nil).Maybe()
|
||||
wbManager.EXPECT().RemoveChannel(mock.Anything).Maybe()
|
||||
|
||||
func getWatchInfoByOpID(opID UniqueID, channel string, state datapb.ChannelWatchState) *datapb.ChannelWatchInfo {
|
||||
return &datapb.ChannelWatchInfo{
|
||||
OpID: opID,
|
||||
State: state,
|
||||
Vchan: &datapb.VchannelInfo{
|
||||
CollectionID: 1,
|
||||
ChannelName: channel,
|
||||
},
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
mockedBroker := &broker.MockBroker{}
|
||||
mockedBroker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).Return([]*datapb.SegmentInfo{}, nil).Maybe()
|
||||
|
||||
s.pipelineParams = &util.PipelineParams{
|
||||
Ctx: context.TODO(),
|
||||
Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 0}},
|
||||
WriteBufferManager: wbManager,
|
||||
Broker: mockedBroker,
|
||||
MsgStreamFactory: factory,
|
||||
DispClient: msgdispatcher.NewClient(factory, typeutil.DataNodeRole, paramtable.GetNodeID()),
|
||||
SyncMgr: syncmgr.NewMockSyncManager(s.T()),
|
||||
Allocator: allocator.NewMockAllocator(s.T()),
|
||||
}
|
||||
|
||||
s.manager = NewChannelManager(s.pipelineParams, pipeline.NewFlowgraphManager())
|
||||
}
|
||||
|
||||
func (s *ChannelManagerSuite) TearDownTest() {
|
||||
@ -165,7 +189,7 @@ func (s *ChannelManagerSuite) TestReleaseStuck() {
|
||||
stuckSig <- struct{}{}
|
||||
}
|
||||
|
||||
info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
|
||||
info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
|
||||
s.Require().Equal(0, s.manager.opRunners.Len())
|
||||
err := s.manager.Submit(info)
|
||||
s.Require().NoError(err)
|
||||
@ -175,9 +199,9 @@ func (s *ChannelManagerSuite) TestReleaseStuck() {
|
||||
|
||||
s.manager.handleOpState(opState)
|
||||
|
||||
releaseInfo := getWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease)
|
||||
paramtable.Get().Save(Params.DataCoordCfg.WatchTimeoutInterval.Key, "0.1")
|
||||
defer paramtable.Get().Reset(Params.DataCoordCfg.WatchTimeoutInterval.Key)
|
||||
releaseInfo := util.GetWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease)
|
||||
paramtable.Get().Save(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key, "0.1")
|
||||
defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key)
|
||||
|
||||
err = s.manager.Submit(releaseInfo)
|
||||
s.NoError(err)
|
||||
@ -201,7 +225,7 @@ func (s *ChannelManagerSuite) TestReleaseStuck() {
|
||||
func (s *ChannelManagerSuite) TestSubmitIdempotent() {
|
||||
channel := "by-dev-rootcoord-dml-1"
|
||||
|
||||
info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
|
||||
info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
|
||||
s.Require().Equal(0, s.manager.opRunners.Len())
|
||||
|
||||
for i := 0; i < 10; i++ {
|
||||
@ -220,7 +244,7 @@ func (s *ChannelManagerSuite) TestSubmitIdempotent() {
|
||||
func (s *ChannelManagerSuite) TestSubmitSkip() {
|
||||
channel := "by-dev-rootcoord-dml-1"
|
||||
|
||||
info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
|
||||
info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
|
||||
s.Require().Equal(0, s.manager.opRunners.Len())
|
||||
|
||||
err := s.manager.Submit(info)
|
||||
@ -232,7 +256,7 @@ func (s *ChannelManagerSuite) TestSubmitSkip() {
|
||||
s.NotNil(opState)
|
||||
s.Equal(datapb.ChannelWatchState_WatchSuccess, opState.state)
|
||||
s.NotNil(opState.fg)
|
||||
s.Equal(info.GetOpID(), opState.fg.opID)
|
||||
s.Equal(info.GetOpID(), opState.fg.GetOpID())
|
||||
s.manager.handleOpState(opState)
|
||||
|
||||
err = s.manager.Submit(info)
|
||||
@ -247,7 +271,7 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() {
|
||||
channel := "by-dev-rootcoord-dml-0"
|
||||
|
||||
// watch
|
||||
info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
|
||||
info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
|
||||
err := s.manager.Submit(info)
|
||||
s.NoError(err)
|
||||
|
||||
@ -256,7 +280,7 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() {
|
||||
s.NotNil(opState)
|
||||
s.Equal(datapb.ChannelWatchState_WatchSuccess, opState.state)
|
||||
s.NotNil(opState.fg)
|
||||
s.Equal(info.GetOpID(), opState.fg.opID)
|
||||
s.Equal(info.GetOpID(), opState.fg.GetOpID())
|
||||
|
||||
resp := s.manager.GetProgress(info)
|
||||
s.Equal(info.GetOpID(), resp.GetOpID())
|
||||
@ -272,7 +296,7 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() {
|
||||
s.Equal(datapb.ChannelWatchState_WatchSuccess, resp.GetState())
|
||||
|
||||
// release
|
||||
info = getWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease)
|
||||
info = util.GetWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease)
|
||||
err = s.manager.Submit(info)
|
||||
s.NoError(err)
|
||||
|
@ -1,6 +1,6 @@
|
||||
// Code generated by mockery v2.32.4. DO NOT EDIT.
|
||||
|
||||
package datanode
|
||||
package channel
|
||||
|
||||
import (
|
||||
datapb "github.com/milvus-io/milvus/internal/proto/datapb"
|
@ -30,7 +30,6 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/io"
|
||||
"github.com/milvus-io/milvus/internal/datanode/metacache"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
@ -405,7 +404,7 @@ func (t *LevelZeroCompactionTask) loadBF(ctx context.Context, targetSegments []*
|
||||
future := pool.Submit(func() (any, error) {
|
||||
_ = binlog.DecompressBinLog(storage.StatsBinlog, segment.GetCollectionID(),
|
||||
segment.GetPartitionID(), segment.GetSegmentID(), segment.GetField2StatslogPaths())
|
||||
pks, err := util.LoadStats(innerCtx, t.cm,
|
||||
pks, err := LoadStats(innerCtx, t.cm,
|
||||
t.plan.GetSchema(), segment.GetSegmentID(), segment.GetField2StatslogPaths())
|
||||
if err != nil {
|
||||
log.Warn("failed to load segment stats log",
|
||||
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package util
|
||||
package compaction
|
||||
|
||||
import (
|
||||
"context"
|
@ -36,9 +36,12 @@ import (
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/broker"
|
||||
"github.com/milvus-io/milvus/internal/datanode/channel"
|
||||
"github.com/milvus-io/milvus/internal/datanode/compaction"
|
||||
"github.com/milvus-io/milvus/internal/datanode/importv2"
|
||||
"github.com/milvus-io/milvus/internal/datanode/pipeline"
|
||||
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/datanode/writebuffer"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
@ -52,7 +55,6 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/util/conc"
|
||||
"github.com/milvus-io/milvus/pkg/util/expr"
|
||||
"github.com/milvus-io/milvus/pkg/util/logutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/retry"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
@ -63,8 +65,6 @@ const (
|
||||
ConnectEtcdMaxRetryTime = 100
|
||||
)
|
||||
|
||||
var getFlowGraphServiceAttempts = uint(50)
|
||||
|
||||
// makes sure DataNode implements types.DataNode
|
||||
var _ types.DataNode = (*DataNode)(nil)
|
||||
|
||||
@ -88,21 +88,19 @@ type DataNode struct {
|
||||
cancel context.CancelFunc
|
||||
Role string
|
||||
stateCode atomic.Value // commonpb.StateCode_Initializing
|
||||
flowgraphManager FlowgraphManager
|
||||
flowgraphManager pipeline.FlowgraphManager
|
||||
|
||||
eventManager *EventManager
|
||||
channelManager ChannelManager
|
||||
channelManager channel.ChannelManager
|
||||
|
||||
syncMgr syncmgr.SyncManager
|
||||
writeBufferManager writebuffer.BufferManager
|
||||
importTaskMgr importv2.TaskManager
|
||||
importScheduler importv2.Scheduler
|
||||
|
||||
clearSignal chan string // vchannel name
|
||||
segmentCache *Cache
|
||||
segmentCache *util.Cache
|
||||
compactionExecutor compaction.Executor
|
||||
timeTickSender *timeTickSender
|
||||
channelCheckpointUpdater *channelCheckpointUpdater
|
||||
timeTickSender *util.TimeTickSender
|
||||
channelCheckpointUpdater *util.ChannelCheckpointUpdater
|
||||
|
||||
etcdCli *clientv3.Client
|
||||
address string
|
||||
@ -114,7 +112,6 @@ type DataNode struct {
|
||||
initOnce sync.Once
|
||||
startOnce sync.Once
|
||||
stopOnce sync.Once
|
||||
stopWaiter sync.WaitGroup
|
||||
sessionMu sync.Mutex // to fix data race
|
||||
session *sessionutil.Session
|
||||
watchKv kv.WatchKV
|
||||
@ -139,14 +136,11 @@ func NewDataNode(ctx context.Context, factory dependency.Factory) *DataNode {
|
||||
cancel: cancel2,
|
||||
Role: typeutil.DataNodeRole,
|
||||
|
||||
rootCoord: nil,
|
||||
dataCoord: nil,
|
||||
factory: factory,
|
||||
segmentCache: newCache(),
|
||||
compactionExecutor: compaction.NewExecutor(),
|
||||
|
||||
clearSignal: make(chan string, 100),
|
||||
|
||||
rootCoord: nil,
|
||||
dataCoord: nil,
|
||||
factory: factory,
|
||||
segmentCache: util.NewCache(),
|
||||
compactionExecutor: compaction.NewExecutor(),
|
||||
reportImportRetryTimes: 10,
|
||||
}
|
||||
node.UpdateStateCode(commonpb.StateCode_Abnormal)
|
||||
@ -215,17 +209,6 @@ func (node *DataNode) initSession() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
// initRateCollector creates and starts rateCollector in QueryNode.
|
||||
func (node *DataNode) initRateCollector() error {
|
||||
err := initGlobalRateCollector()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
rateCol.Register(metricsinfo.InsertConsumeThroughput)
|
||||
rateCol.Register(metricsinfo.DeleteConsumeThroughput)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (node *DataNode) GetNodeID() int64 {
|
||||
if node.session != nil {
|
||||
return node.session.ServerID
|
||||
@ -250,7 +233,7 @@ func (node *DataNode) Init() error {
|
||||
|
||||
node.broker = broker.NewCoordBroker(node.dataCoord, serverID)
|
||||
|
||||
err := node.initRateCollector()
|
||||
err := util.InitGlobalRateCollector()
|
||||
if err != nil {
|
||||
log.Error("DataNode server init rateCollector failed", zap.Error(err))
|
||||
initError = err
|
||||
@ -292,38 +275,14 @@ func (node *DataNode) Init() error {
|
||||
|
||||
node.importTaskMgr = importv2.NewTaskManager()
|
||||
node.importScheduler = importv2.NewScheduler(node.importTaskMgr)
|
||||
node.channelCheckpointUpdater = newChannelCheckpointUpdater(node.broker)
|
||||
node.flowgraphManager = newFlowgraphManager()
|
||||
node.channelCheckpointUpdater = util.NewChannelCheckpointUpdater(node.broker)
|
||||
node.flowgraphManager = pipeline.NewFlowgraphManager()
|
||||
|
||||
if paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.GetAsBool() {
|
||||
node.channelManager = NewChannelManager(node)
|
||||
} else {
|
||||
node.eventManager = NewEventManager()
|
||||
}
|
||||
log.Info("init datanode done", zap.String("Address", node.address))
|
||||
})
|
||||
return initError
|
||||
}
|
||||
|
||||
// handleChannelEvt handles event from kv watch event
|
||||
func (node *DataNode) handleChannelEvt(evt *clientv3.Event) {
|
||||
var e *event
|
||||
switch evt.Type {
|
||||
case clientv3.EventTypePut: // datacoord shall put channels needs to be watched here
|
||||
e = &event{
|
||||
eventType: putEventType,
|
||||
version: evt.Kv.Version,
|
||||
}
|
||||
|
||||
case clientv3.EventTypeDelete:
|
||||
e = &event{
|
||||
eventType: deleteEventType,
|
||||
version: evt.Kv.Version,
|
||||
}
|
||||
}
|
||||
node.handleWatchInfo(e, string(evt.Kv.Key), evt.Kv.Value)
|
||||
}
|
||||
|
||||
// tryToReleaseFlowgraph tries to release a flowgraph
|
||||
func (node *DataNode) tryToReleaseFlowgraph(channel string) {
|
||||
log.Info("try to release flowgraph", zap.String("channel", channel))
|
||||
@ -338,22 +297,6 @@ func (node *DataNode) tryToReleaseFlowgraph(channel string) {
|
||||
}
|
||||
}
|
||||
|
||||
// BackGroundGC runs in background to release datanode resources
|
||||
// GOOSE TODO: remove background GC, using ToRelease for drop-collection after #15846
|
||||
func (node *DataNode) BackGroundGC(vChannelCh <-chan string) {
|
||||
defer node.stopWaiter.Done()
|
||||
log.Info("DataNode Background GC Start")
|
||||
for {
|
||||
select {
|
||||
case vchanName := <-vChannelCh:
|
||||
node.tryToReleaseFlowgraph(vchanName)
|
||||
case <-node.ctx.Done():
|
||||
log.Warn("DataNode context done, exiting background GC")
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Start will update DataNode state to HEALTHY
|
||||
func (node *DataNode) Start() error {
|
||||
var startErr error
|
||||
@ -365,21 +308,6 @@ func (node *DataNode) Start() error {
|
||||
}
|
||||
log.Info("start id allocator done", zap.String("role", typeutil.DataNodeRole))
|
||||
|
||||
/*
|
||||
rep, err := node.rootCoord.AllocTimestamp(node.ctx, &rootcoordpb.AllocTimestampRequest{
|
||||
Base: commonpbutil.NewMsgBase(
|
||||
commonpbutil.WithMsgType(commonpb.MsgType_RequestTSO),
|
||||
commonpbutil.WithMsgID(0),
|
||||
commonpbutil.WithSourceID(node.GetNodeID()),
|
||||
),
|
||||
Count: 1,
|
||||
})
|
||||
if err != nil || rep.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
|
||||
log.Warn("fail to alloc timestamp", zap.Any("rep", rep), zap.Error(err))
|
||||
startErr = errors.New("DataNode fail to alloc timestamp")
|
||||
return
|
||||
}*/
|
||||
|
||||
connectEtcdFn := func() error {
|
||||
etcdKV := etcdkv.NewEtcdKV(node.etcdCli, Params.EtcdCfg.MetaRootPath.GetValue(),
|
||||
etcdkv.WithRequestTimeout(paramtable.Get().ServiceParam.EtcdCfg.RequestTimeout.GetAsDuration(time.Millisecond)))
|
||||
@ -394,27 +322,20 @@ func (node *DataNode) Start() error {
|
||||
|
||||
node.writeBufferManager.Start()
|
||||
|
||||
node.stopWaiter.Add(1)
|
||||
go node.BackGroundGC(node.clearSignal)
|
||||
|
||||
go node.compactionExecutor.Start(node.ctx)
|
||||
|
||||
go node.importScheduler.Start()
|
||||
|
||||
if Params.DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() {
|
||||
node.timeTickSender = newTimeTickSender(node.broker, node.session.ServerID,
|
||||
node.timeTickSender = util.NewTimeTickSender(node.broker, node.session.ServerID,
|
||||
retry.Attempts(20), retry.Sleep(time.Millisecond*100))
|
||||
node.timeTickSender.start()
|
||||
node.timeTickSender.Start()
|
||||
}
|
||||
|
||||
go node.channelCheckpointUpdater.start()
|
||||
go node.channelCheckpointUpdater.Start()
|
||||
|
||||
if paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.GetAsBool() {
|
||||
node.channelManager.Start()
|
||||
} else {
|
||||
// Start node watch node
|
||||
node.startWatchChannelsAtBackground(node.ctx)
|
||||
}
|
||||
node.channelManager = channel.NewChannelManager(getPipelineParams(node), node.flowgraphManager)
|
||||
node.channelManager.Start()
|
||||
|
||||
node.UpdateStateCode(commonpb.StateCode_Healthy)
|
||||
})
|
||||
@ -452,10 +373,6 @@ func (node *DataNode) Stop() error {
|
||||
node.channelManager.Close()
|
||||
}
|
||||
|
||||
if node.eventManager != nil {
|
||||
node.eventManager.CloseAll()
|
||||
}
|
||||
|
||||
if node.writeBufferManager != nil {
|
||||
node.writeBufferManager.Stop()
|
||||
}
|
||||
@ -478,7 +395,7 @@ func (node *DataNode) Stop() error {
|
||||
}
|
||||
|
||||
if node.channelCheckpointUpdater != nil {
|
||||
node.channelCheckpointUpdater.close()
|
||||
node.channelCheckpointUpdater.Close()
|
||||
}
|
||||
|
||||
if node.importScheduler != nil {
|
||||
@ -487,21 +404,37 @@ func (node *DataNode) Stop() error {
|
||||
|
||||
// Delay the cancellation of ctx to ensure that the session is automatically recycled after closed the flow graph
|
||||
node.cancel()
|
||||
node.stopWaiter.Wait()
|
||||
})
|
||||
return nil
|
||||
}
|
||||
|
||||
// to fix data race
|
||||
// SetSession to fix data race
|
||||
func (node *DataNode) SetSession(session *sessionutil.Session) {
|
||||
node.sessionMu.Lock()
|
||||
defer node.sessionMu.Unlock()
|
||||
node.session = session
|
||||
}
|
||||
|
||||
// to fix data race
|
||||
// GetSession to fix data race
|
||||
func (node *DataNode) GetSession() *sessionutil.Session {
|
||||
node.sessionMu.Lock()
|
||||
defer node.sessionMu.Unlock()
|
||||
return node.session
|
||||
}
|
||||
|
||||
func getPipelineParams(node *DataNode) *util.PipelineParams {
|
||||
return &util.PipelineParams{
|
||||
Ctx: node.ctx,
|
||||
Broker: node.broker,
|
||||
SyncMgr: node.syncMgr,
|
||||
TimeTickSender: node.timeTickSender,
|
||||
CompactionExecutor: node.compactionExecutor,
|
||||
MsgStreamFactory: node.factory,
|
||||
DispClient: node.dispClient,
|
||||
ChunkManager: node.chunkManager,
|
||||
Session: node.session,
|
||||
WriteBufferManager: node.writeBufferManager,
|
||||
CheckpointUpdater: node.channelCheckpointUpdater,
|
||||
Allocator: node.allocator,
|
||||
}
|
||||
}
|
||||
|
@ -32,21 +32,23 @@ import (
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/broker"
|
||||
"github.com/milvus-io/milvus/internal/datanode/pipeline"
|
||||
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/datanode/writebuffer"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
||||
"github.com/milvus-io/milvus/pkg/util/etcd"
|
||||
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
const returnError = "ReturnError"
|
||||
|
||||
type ctxKey struct{}
|
||||
|
||||
func TestMain(t *testing.M) {
|
||||
rand.Seed(time.Now().Unix())
|
||||
// init embed etcd
|
||||
@ -70,7 +72,7 @@ func TestMain(t *testing.M) {
|
||||
paramtable.Get().Save(Params.EtcdCfg.Endpoints.Key, strings.Join(addrs, ","))
|
||||
paramtable.Get().Save(Params.CommonCfg.DataCoordTimeTick.Key, Params.CommonCfg.DataCoordTimeTick.GetValue()+strconv.Itoa(rand.Int()))
|
||||
|
||||
rateCol, err = newRateCollector()
|
||||
err = util.InitGlobalRateCollector()
|
||||
if err != nil {
|
||||
panic("init test failed, err = " + err.Error())
|
||||
}
|
||||
@ -79,11 +81,31 @@ func TestMain(t *testing.M) {
|
||||
os.Exit(code)
|
||||
}
|
||||
|
||||
func NewIDLEDataNodeMock(ctx context.Context, pkType schemapb.DataType) *DataNode {
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
node := NewDataNode(ctx, factory)
|
||||
node.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}})
|
||||
node.dispClient = msgdispatcher.NewClient(factory, typeutil.DataNodeRole, paramtable.GetNodeID())
|
||||
|
||||
broker := &broker.MockBroker{}
|
||||
broker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
broker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).Return([]*datapb.SegmentInfo{}, nil).Maybe()
|
||||
|
||||
node.broker = broker
|
||||
node.timeTickSender = util.NewTimeTickSender(broker, 0)
|
||||
|
||||
syncMgr, _ := syncmgr.NewSyncManager(node.chunkManager, node.allocator)
|
||||
|
||||
node.syncMgr = syncMgr
|
||||
node.writeBufferManager = writebuffer.NewManager(syncMgr)
|
||||
|
||||
return node
|
||||
}
|
||||
|
||||
func TestDataNode(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
node := newIDLEDataNodeMock(ctx, schemapb.DataType_Int64)
|
||||
node := NewIDLEDataNodeMock(ctx, schemapb.DataType_Int64)
|
||||
etcdCli, err := etcd.GetEtcdClient(
|
||||
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
|
||||
Params.EtcdCfg.EtcdUseSSL.GetAsBool(),
|
||||
@ -123,7 +145,7 @@ func TestDataNode(t *testing.T) {
|
||||
description string
|
||||
}{
|
||||
{nil, false, "nil input"},
|
||||
{&RootCoordFactory{}, true, "valid input"},
|
||||
{&util.RootCoordFactory{}, true, "valid input"},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
@ -146,7 +168,7 @@ func TestDataNode(t *testing.T) {
|
||||
description string
|
||||
}{
|
||||
{nil, false, "nil input"},
|
||||
{&DataCoordFactory{}, true, "valid input"},
|
||||
{&util.DataCoordFactory{}, true, "valid input"},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
@ -164,7 +186,7 @@ func TestDataNode(t *testing.T) {
|
||||
t.Run("Test getSystemInfoMetrics", func(t *testing.T) {
|
||||
emptyNode := &DataNode{}
|
||||
emptyNode.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}})
|
||||
emptyNode.flowgraphManager = newFlowgraphManager()
|
||||
emptyNode.flowgraphManager = pipeline.NewFlowgraphManager()
|
||||
|
||||
req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics)
|
||||
assert.NoError(t, err)
|
||||
@ -179,64 +201,14 @@ func TestDataNode(t *testing.T) {
|
||||
t.Run("Test getSystemInfoMetrics with quotaMetric error", func(t *testing.T) {
|
||||
emptyNode := &DataNode{}
|
||||
emptyNode.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}})
|
||||
emptyNode.flowgraphManager = newFlowgraphManager()
|
||||
emptyNode.flowgraphManager = pipeline.NewFlowgraphManager()
|
||||
|
||||
req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics)
|
||||
assert.NoError(t, err)
|
||||
rateCol.Deregister(metricsinfo.InsertConsumeThroughput)
|
||||
util.DeregisterRateCollector(metricsinfo.InsertConsumeThroughput)
|
||||
resp, err := emptyNode.getSystemInfoMetrics(context.TODO(), req)
|
||||
assert.NoError(t, err)
|
||||
assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
|
||||
rateCol.Register(metricsinfo.InsertConsumeThroughput)
|
||||
})
|
||||
|
||||
t.Run("Test BackGroundGC", func(t *testing.T) {
|
||||
vchanNameCh := make(chan string)
|
||||
node.clearSignal = vchanNameCh
|
||||
node.stopWaiter.Add(1)
|
||||
go node.BackGroundGC(vchanNameCh)
|
||||
|
||||
testDataSyncs := []struct {
|
||||
dmChannelName string
|
||||
}{
|
||||
{"fake-by-dev-rootcoord-dml-backgroundgc-1"},
|
||||
{"fake-by-dev-rootcoord-dml-backgroundgc-2"},
|
||||
}
|
||||
|
||||
for _, test := range testDataSyncs {
|
||||
err = node.flowgraphManager.AddandStartWithEtcdTickler(node, &datapb.VchannelInfo{
|
||||
CollectionID: 1, ChannelName: test.dmChannelName,
|
||||
}, &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, genTestTickler())
|
||||
assert.NoError(t, err)
|
||||
vchanNameCh <- test.dmChannelName
|
||||
}
|
||||
|
||||
assert.Eventually(t, func() bool {
|
||||
for _, test := range testDataSyncs {
|
||||
if node.flowgraphManager.HasFlowgraph(test.dmChannelName) {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}, 2*time.Second, 10*time.Millisecond)
|
||||
util.RegisterRateCollector(metricsinfo.InsertConsumeThroughput)
|
||||
})
|
||||
}
|
||||
|
@ -1,447 +0,0 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"path"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
v3rpc "go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
|
||||
"go.uber.org/atomic"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/kv"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/logutil"
|
||||
)
|
||||
|
||||
const retryWatchInterval = 20 * time.Second
|
||||
|
||||
func (node *DataNode) startWatchChannelsAtBackground(ctx context.Context) {
|
||||
node.stopWaiter.Add(1)
|
||||
go node.StartWatchChannels(ctx)
|
||||
}
|
||||
|
||||
// StartWatchChannels start loop to watch channel allocation status via kv(etcd for now)
|
||||
func (node *DataNode) StartWatchChannels(ctx context.Context) {
|
||||
defer node.stopWaiter.Done()
|
||||
defer logutil.LogPanic()
|
||||
// REF MEP#7 watch path should be [prefix]/channel/{node_id}/{channel_name}
|
||||
// TODO, this is risky, we'd better watch etcd with revision rather simply a path
|
||||
watchPrefix := path.Join(Params.CommonCfg.DataCoordWatchSubPath.GetValue(), fmt.Sprintf("%d", node.GetSession().ServerID))
|
||||
log.Info("Start watch channel", zap.String("prefix", watchPrefix))
|
||||
evtChan := node.watchKv.WatchWithPrefix(watchPrefix)
|
||||
// after watch, first check all exists nodes first
|
||||
err := node.checkWatchedList()
|
||||
if err != nil {
|
||||
log.Warn("StartWatchChannels failed", zap.Error(err))
|
||||
return
|
||||
}
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
log.Info("watch etcd loop quit")
|
||||
return
|
||||
case event, ok := <-evtChan:
|
||||
if !ok {
|
||||
log.Warn("datanode failed to watch channel, return")
|
||||
node.startWatchChannelsAtBackground(ctx)
|
||||
return
|
||||
}
|
||||
|
||||
if err := event.Err(); err != nil {
|
||||
log.Warn("datanode watch channel canceled", zap.Error(event.Err()))
|
||||
// https://github.com/etcd-io/etcd/issues/8980
|
||||
if event.Err() == v3rpc.ErrCompacted {
|
||||
node.startWatchChannelsAtBackground(ctx)
|
||||
return
|
||||
}
|
||||
// if watch loop return due to event canceled, the datanode is not functional anymore
|
||||
log.Panic("datanode is not functional for event canceled", zap.Error(err))
|
||||
return
|
||||
}
|
||||
for _, evt := range event.Events {
|
||||
// We need to stay in order until events enqueued
|
||||
node.handleChannelEvt(evt)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// checkWatchedList list all nodes under [prefix]/channel/{node_id} and make sure all nodes are watched
|
||||
// serves the corner case for etcd connection lost and missing some events
|
||||
func (node *DataNode) checkWatchedList() error {
|
||||
// REF MEP#7 watch path should be [prefix]/channel/{node_id}/{channel_name}
|
||||
prefix := path.Join(Params.CommonCfg.DataCoordWatchSubPath.GetValue(), fmt.Sprintf("%d", node.GetNodeID()))
|
||||
keys, values, err := node.watchKv.LoadWithPrefix(prefix)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for i, val := range values {
|
||||
node.handleWatchInfo(&event{eventType: putEventType}, keys[i], []byte(val))
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (node *DataNode) handleWatchInfo(e *event, key string, data []byte) {
|
||||
switch e.eventType {
|
||||
case putEventType:
|
||||
watchInfo, err := parsePutEventData(data)
|
||||
if err != nil {
|
||||
log.Warn("fail to handle watchInfo", zap.Int("event type", e.eventType), zap.String("key", key), zap.Error(err))
|
||||
return
|
||||
}
|
||||
|
||||
if isEndWatchState(watchInfo.State) {
|
||||
log.Info("DataNode received a PUT event with an end State", zap.String("state", watchInfo.State.String()))
|
||||
return
|
||||
}
|
||||
|
||||
if watchInfo.Progress != 0 {
|
||||
log.Info("DataNode received a PUT event with tickler update progress", zap.String("channel", watchInfo.Vchan.ChannelName), zap.Int64("version", e.version))
|
||||
return
|
||||
}
|
||||
|
||||
e.info = watchInfo
|
||||
e.vChanName = watchInfo.GetVchan().GetChannelName()
|
||||
log.Info("DataNode is handling watchInfo PUT event", zap.String("key", key), zap.String("watch state", watchInfo.GetState().String()))
|
||||
case deleteEventType:
|
||||
e.vChanName = parseDeleteEventKey(key)
|
||||
log.Info("DataNode is handling watchInfo DELETE event", zap.String("key", key))
|
||||
}
|
||||
|
||||
actualManager := node.eventManager.GetOrInsert(e.vChanName, newChannelEventManager(
|
||||
node.handlePutEvent, node.handleDeleteEvent, retryWatchInterval,
|
||||
))
|
||||
|
||||
actualManager.handleEvent(*e)
|
||||
|
||||
// Whenever a delete event comes, this eventManager will be removed from map
|
||||
if e.eventType == deleteEventType {
|
||||
node.eventManager.Remove(e.vChanName)
|
||||
}
|
||||
}
|
||||
|
||||
func parsePutEventData(data []byte) (*datapb.ChannelWatchInfo, error) {
|
||||
watchInfo := datapb.ChannelWatchInfo{}
|
||||
err := proto.Unmarshal(data, &watchInfo)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("invalid event data: fail to parse ChannelWatchInfo, err: %v", err)
|
||||
}
|
||||
|
||||
if watchInfo.Vchan == nil {
|
||||
return nil, fmt.Errorf("invalid event: ChannelWatchInfo with nil VChannelInfo")
|
||||
}
|
||||
reviseVChannelInfo(watchInfo.GetVchan())
|
||||
return &watchInfo, nil
|
||||
}
|
||||
|
||||
func parseDeleteEventKey(key string) string {
|
||||
parts := strings.Split(key, "/")
|
||||
vChanName := parts[len(parts)-1]
|
||||
return vChanName
|
||||
}
|
||||
|
||||
func (node *DataNode) handlePutEvent(watchInfo *datapb.ChannelWatchInfo, version int64) (err error) {
|
||||
vChanName := watchInfo.GetVchan().GetChannelName()
|
||||
key := path.Join(Params.CommonCfg.DataCoordWatchSubPath.GetValue(), fmt.Sprintf("%d", node.GetSession().ServerID), vChanName)
|
||||
tickler := newEtcdTickler(version, key, watchInfo, node.watchKv, Params.DataNodeCfg.WatchEventTicklerInterval.GetAsDuration(time.Second))
|
||||
|
||||
switch watchInfo.State {
|
||||
case datapb.ChannelWatchState_Uncomplete, datapb.ChannelWatchState_ToWatch:
|
||||
if err := node.flowgraphManager.AddandStartWithEtcdTickler(node, watchInfo.GetVchan(), watchInfo.GetSchema(), tickler); err != nil {
|
||||
log.Warn("handle put event: new data sync service failed", zap.String("vChanName", vChanName), zap.Error(err))
|
||||
watchInfo.State = datapb.ChannelWatchState_WatchFailure
|
||||
} else {
|
||||
log.Info("handle put event: new data sync service success", zap.String("vChanName", vChanName))
|
||||
watchInfo.State = datapb.ChannelWatchState_WatchSuccess
|
||||
}
|
||||
case datapb.ChannelWatchState_ToRelease:
|
||||
// there is no reason why we release fail
|
||||
node.tryToReleaseFlowgraph(vChanName)
|
||||
watchInfo.State = datapb.ChannelWatchState_ReleaseSuccess
|
||||
}
|
||||
|
||||
liteInfo := GetLiteChannelWatchInfo(watchInfo)
|
||||
v, err := proto.Marshal(liteInfo)
|
||||
if err != nil {
|
||||
return fmt.Errorf("fail to marshal watchInfo with state, vChanName: %s, state: %s ,err: %w", vChanName, liteInfo.State.String(), err)
|
||||
}
|
||||
|
||||
success, err := node.watchKv.CompareVersionAndSwap(key, tickler.version, string(v))
|
||||
// etcd error
|
||||
if err != nil {
|
||||
// flow graph will leak if not release, causing new datanode failed to subscribe
|
||||
node.tryToReleaseFlowgraph(vChanName)
|
||||
log.Warn("fail to update watch state to etcd", zap.String("vChanName", vChanName),
|
||||
zap.String("state", watchInfo.State.String()), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
// etcd valid but the states updated.
|
||||
if !success {
|
||||
log.Info("handle put event: failed to compare version and swap, release flowgraph",
|
||||
zap.String("key", key), zap.String("state", watchInfo.State.String()),
|
||||
zap.String("vChanName", vChanName))
|
||||
// flow graph will leak if not release, causing new datanode failed to subscribe
|
||||
node.tryToReleaseFlowgraph(vChanName)
|
||||
return nil
|
||||
}
|
||||
log.Info("handle put event success", zap.String("key", key),
|
||||
zap.String("state", watchInfo.State.String()), zap.String("vChanName", vChanName))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (node *DataNode) handleDeleteEvent(vChanName string) {
|
||||
node.tryToReleaseFlowgraph(vChanName)
|
||||
}
|
||||
|
||||
// event represents a single event with specified channel and version.
|
||||
type event struct {
|
||||
eventType int
|
||||
vChanName string
|
||||
version int64
|
||||
info *datapb.ChannelWatchInfo
|
||||
}
|
||||
|
||||
// channelEventManager is used to handle events from channel watched event.
|
||||
type channelEventManager struct {
|
||||
sync.Once
|
||||
wg sync.WaitGroup
|
||||
eventChan chan event
|
||||
closeChan chan struct{}
|
||||
handlePutEvent func(watchInfo *datapb.ChannelWatchInfo, version int64) error // node.handlePutEvent
|
||||
handleDeleteEvent func(vChanName string) // node.handleDeleteEvent
|
||||
retryInterval time.Duration
|
||||
}
|
||||
|
||||
// These are valid types of an event.
|
||||
const (
|
||||
putEventType = 1
|
||||
deleteEventType = 2
|
||||
)
|
||||
|
||||
func newChannelEventManager(handlePut func(*datapb.ChannelWatchInfo, int64) error,
|
||||
handleDel func(string), retryInterval time.Duration,
|
||||
) *channelEventManager {
|
||||
return &channelEventManager{
|
||||
eventChan: make(chan event, 10),
|
||||
closeChan: make(chan struct{}),
|
||||
handlePutEvent: handlePut,
|
||||
handleDeleteEvent: handleDel,
|
||||
retryInterval: retryInterval,
|
||||
}
|
||||
}
|
||||
|
||||
func (e *channelEventManager) Run() {
|
||||
e.wg.Add(1)
|
||||
go func() {
|
||||
defer e.wg.Done()
|
||||
for {
|
||||
select {
|
||||
case event := <-e.eventChan:
|
||||
switch event.eventType {
|
||||
case putEventType:
|
||||
err := e.handlePutEvent(event.info, event.version)
|
||||
if err != nil {
|
||||
// logging the error is convenient for follow-up investigation of problems
|
||||
log.Warn("handle put event failed", zap.String("vChanName", event.vChanName), zap.Error(err))
|
||||
}
|
||||
case deleteEventType:
|
||||
e.handleDeleteEvent(event.vChanName)
|
||||
}
|
||||
case <-e.closeChan:
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
func (e *channelEventManager) handleEvent(event event) {
|
||||
e.eventChan <- event
|
||||
}
|
||||
|
||||
func (e *channelEventManager) Close() {
|
||||
e.Do(func() {
|
||||
close(e.closeChan)
|
||||
e.wg.Wait()
|
||||
})
|
||||
}
|
||||
|
||||
func isEndWatchState(state datapb.ChannelWatchState) bool {
|
||||
return state != datapb.ChannelWatchState_ToWatch && // start watch
|
||||
state != datapb.ChannelWatchState_ToRelease && // start release
|
||||
state != datapb.ChannelWatchState_Uncomplete // legacy state, equal to ToWatch
|
||||
}
|
||||
|
||||
type etcdTickler struct {
|
||||
progress *atomic.Int32
|
||||
version int64
|
||||
|
||||
kv kv.WatchKV
|
||||
path string
|
||||
watchInfo *datapb.ChannelWatchInfo
|
||||
|
||||
interval time.Duration
|
||||
closeCh chan struct{}
|
||||
closeWg sync.WaitGroup
|
||||
isWatchFailed *atomic.Bool
|
||||
}
|
||||
|
||||
func (t *etcdTickler) inc() {
|
||||
t.progress.Inc()
|
||||
}
|
||||
|
||||
func (t *etcdTickler) watch() {
|
||||
if t.interval == 0 {
|
||||
log.Info("zero interval, close ticler watch",
|
||||
zap.String("channelName", t.watchInfo.GetVchan().GetChannelName()),
|
||||
)
|
||||
return
|
||||
}
|
||||
|
||||
t.closeWg.Add(1)
|
||||
go func() {
|
||||
defer t.closeWg.Done()
|
||||
ticker := time.NewTicker(t.interval)
|
||||
defer ticker.Stop()
|
||||
for {
|
||||
select {
|
||||
case <-ticker.C:
|
||||
nowProgress := t.progress.Load()
|
||||
if t.watchInfo.Progress == nowProgress {
|
||||
continue
|
||||
}
|
||||
|
||||
t.watchInfo.Progress = nowProgress
|
||||
v, err := proto.Marshal(t.watchInfo)
|
||||
if err != nil {
|
||||
log.Error("fail to marshal watchInfo with progress at tickler",
|
||||
zap.String("vChanName", t.watchInfo.Vchan.ChannelName),
|
||||
zap.Int32("progree", nowProgress),
|
||||
zap.Error(err))
|
||||
t.isWatchFailed.Store(true)
|
||||
return
|
||||
}
|
||||
success, err := t.kv.CompareVersionAndSwap(t.path, t.version, string(v))
|
||||
if err != nil {
|
||||
log.Error("tickler update failed", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
if !success {
|
||||
log.Error("tickler update failed: failed to compare version and swap",
|
||||
zap.String("key", t.path), zap.Int32("progress", nowProgress), zap.Int64("version", t.version),
|
||||
zap.String("vChanName", t.watchInfo.GetVchan().ChannelName))
|
||||
t.isWatchFailed.Store(true)
|
||||
return
|
||||
}
|
||||
log.Debug("tickler update success", zap.Int32("progress", nowProgress), zap.Int64("version", t.version),
|
||||
zap.String("vChanName", t.watchInfo.GetVchan().ChannelName))
|
||||
t.version++
|
||||
case <-t.closeCh:
|
||||
return
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
func (t *etcdTickler) stop() {
|
||||
close(t.closeCh)
|
||||
t.closeWg.Wait()
|
||||
}
|
||||
|
||||
func newEtcdTickler(version int64, path string, watchInfo *datapb.ChannelWatchInfo, kv kv.WatchKV, interval time.Duration) *etcdTickler {
|
||||
liteWatchInfo := GetLiteChannelWatchInfo(watchInfo)
|
||||
return &etcdTickler{
|
||||
progress: atomic.NewInt32(0),
|
||||
path: path,
|
||||
kv: kv,
|
||||
watchInfo: liteWatchInfo,
|
||||
version: version,
|
||||
interval: interval,
|
||||
closeCh: make(chan struct{}),
|
||||
isWatchFailed: atomic.NewBool(false),
|
||||
}
|
||||
}
|
||||
|
||||
// GetLiteChannelWatchInfo clones watchInfo without segmentIDs to reduce the size of the message
|
||||
func GetLiteChannelWatchInfo(watchInfo *datapb.ChannelWatchInfo) *datapb.ChannelWatchInfo {
|
||||
return &datapb.ChannelWatchInfo{
|
||||
Vchan: &datapb.VchannelInfo{
|
||||
CollectionID: watchInfo.GetVchan().GetCollectionID(),
|
||||
ChannelName: watchInfo.GetVchan().GetChannelName(),
|
||||
SeekPosition: watchInfo.GetVchan().GetSeekPosition(),
|
||||
},
|
||||
StartTs: watchInfo.GetStartTs(),
|
||||
State: watchInfo.GetState(),
|
||||
TimeoutTs: watchInfo.GetTimeoutTs(),
|
||||
Schema: watchInfo.GetSchema(),
|
||||
Progress: watchInfo.GetProgress(),
|
||||
}
|
||||
}
|
||||
|
||||
type EventManager struct {
|
||||
channelGuard sync.Mutex
|
||||
channelManagers map[string]*channelEventManager
|
||||
}
|
||||
|
||||
func NewEventManager() *EventManager {
|
||||
return &EventManager{
|
||||
channelManagers: make(map[string]*channelEventManager),
|
||||
}
|
||||
}
|
||||
|
||||
func (m *EventManager) GetOrInsert(channel string, newManager *channelEventManager) *channelEventManager {
|
||||
m.channelGuard.Lock()
|
||||
defer m.channelGuard.Unlock()
|
||||
|
||||
eManager, got := m.channelManagers[channel]
|
||||
if !got {
|
||||
newManager.Run()
|
||||
m.channelManagers[channel] = newManager
|
||||
return newManager
|
||||
}
|
||||
|
||||
return eManager
|
||||
}
|
||||
|
||||
func (m *EventManager) Remove(channel string) {
|
||||
m.channelGuard.Lock()
|
||||
eManager, got := m.channelManagers[channel]
|
||||
delete(m.channelManagers, channel)
|
||||
m.channelGuard.Unlock()
|
||||
|
||||
if got {
|
||||
eManager.Close()
|
||||
}
|
||||
}
|
||||
|
||||
func (m *EventManager) CloseAll() {
|
||||
m.channelGuard.Lock()
|
||||
defer m.channelGuard.Unlock()
|
||||
|
||||
for channel, eManager := range m.channelManagers {
|
||||
delete(m.channelManagers, channel)
|
||||
eManager.Close()
|
||||
}
|
||||
}
|
@ -1,626 +0,0 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"path"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/broker"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/util/etcd"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
)
|
||||
|
||||
func TestWatchChannel(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
paramtable.Get().Save(paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.Key, "false")
|
||||
defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.Key)
|
||||
|
||||
node := newIDLEDataNodeMock(ctx, schemapb.DataType_Int64)
|
||||
etcdCli, err := etcd.GetEtcdClient(
|
||||
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
|
||||
Params.EtcdCfg.EtcdUseSSL.GetAsBool(),
|
||||
Params.EtcdCfg.Endpoints.GetAsStrings(),
|
||||
Params.EtcdCfg.EtcdTLSCert.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSKey.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSCACert.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSMinVersion.GetValue())
|
||||
assert.NoError(t, err)
|
||||
defer etcdCli.Close()
|
||||
node.SetEtcdClient(etcdCli)
|
||||
err = node.Init()
|
||||
|
||||
broker := broker.NewMockBroker(t)
|
||||
broker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
broker.EXPECT().SaveBinlogPaths(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
broker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).Return([]*datapb.SegmentInfo{}, nil).Maybe()
|
||||
broker.EXPECT().DropVirtualChannel(mock.Anything, mock.Anything).Return(nil, nil).Maybe()
|
||||
broker.EXPECT().UpdateChannelCheckpoint(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
|
||||
node.broker = broker
|
||||
|
||||
assert.NoError(t, err)
|
||||
err = node.Start()
|
||||
assert.NoError(t, err)
|
||||
defer node.Stop()
|
||||
err = node.Register()
|
||||
assert.NoError(t, err)
|
||||
|
||||
defer cancel()
|
||||
|
||||
t.Run("test watch channel", func(t *testing.T) {
|
||||
kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
|
||||
oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid"
|
||||
path := fmt.Sprintf("%s/%d/%s", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID(), oldInvalidCh)
|
||||
err = kv.Save(path, string([]byte{23}))
|
||||
assert.NoError(t, err)
|
||||
|
||||
ch := fmt.Sprintf("datanode-etcd-test-by-dev-rootcoord-dml-channel_%d", rand.Int31())
|
||||
path = fmt.Sprintf("%s/%d/%s", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID(), ch)
|
||||
|
||||
vchan := &datapb.VchannelInfo{
|
||||
CollectionID: 1,
|
||||
ChannelName: ch,
|
||||
UnflushedSegmentIds: []int64{},
|
||||
}
|
||||
info := &datapb.ChannelWatchInfo{
|
||||
State: datapb.ChannelWatchState_ToWatch,
|
||||
Vchan: vchan,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
val, err := proto.Marshal(info)
|
||||
assert.NoError(t, err)
|
||||
err = kv.Save(path, string(val))
|
||||
assert.NoError(t, err)
|
||||
|
||||
assert.Eventually(t, func() bool {
|
||||
exist := node.flowgraphManager.HasFlowgraph(ch)
|
||||
if !exist {
|
||||
return false
|
||||
}
|
||||
bs, err := kv.LoadBytes(fmt.Sprintf("%s/%d/%s", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID(), ch))
|
||||
if err != nil {
|
||||
return false
|
||||
}
|
||||
watchInfo := &datapb.ChannelWatchInfo{}
|
||||
err = proto.Unmarshal(bs, watchInfo)
|
||||
if err != nil {
|
||||
return false
|
||||
}
|
||||
return watchInfo.GetState() == datapb.ChannelWatchState_WatchSuccess
|
||||
}, 3*time.Second, 100*time.Millisecond)
|
||||
|
||||
err = kv.RemoveWithPrefix(fmt.Sprintf("%s/%d", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID()))
|
||||
assert.NoError(t, err)
|
||||
|
||||
assert.Eventually(t, func() bool {
|
||||
exist := node.flowgraphManager.HasFlowgraph(ch)
|
||||
return !exist
|
||||
}, 3*time.Second, 100*time.Millisecond)
|
||||
})
|
||||
|
||||
t.Run("Test release channel", func(t *testing.T) {
|
||||
kv := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue())
|
||||
oldInvalidCh := "datanode-etcd-test-by-dev-rootcoord-dml-channel-invalid"
|
||||
path := fmt.Sprintf("%s/%d/%s", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID(), oldInvalidCh)
|
||||
err = kv.Save(path, string([]byte{23}))
|
||||
assert.NoError(t, err)
|
||||
|
||||
ch := fmt.Sprintf("datanode-etcd-test-by-dev-rootcoord-dml-channel_%d", rand.Int31())
|
||||
path = fmt.Sprintf("%s/%d/%s", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID(), ch)
|
||||
c := make(chan struct{})
|
||||
go func() {
|
||||
ec := kv.WatchWithPrefix(fmt.Sprintf("%s/%d", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID()))
|
||||
c <- struct{}{}
|
||||
cnt := 0
|
||||
for {
|
||||
evt := <-ec
|
||||
for _, event := range evt.Events {
|
||||
if strings.Contains(string(event.Kv.Key), ch) {
|
||||
cnt++
|
||||
}
|
||||
}
|
||||
if cnt >= 2 {
|
||||
break
|
||||
}
|
||||
}
|
||||
c <- struct{}{}
|
||||
}()
|
||||
// wait for check goroutine start Watch
|
||||
<-c
|
||||
|
||||
vchan := &datapb.VchannelInfo{
|
||||
CollectionID: 1,
|
||||
ChannelName: ch,
|
||||
UnflushedSegmentIds: []int64{},
|
||||
}
|
||||
info := &datapb.ChannelWatchInfo{
|
||||
State: datapb.ChannelWatchState_ToRelease,
|
||||
Vchan: vchan,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
val, err := proto.Marshal(info)
|
||||
assert.NoError(t, err)
|
||||
err = kv.Save(path, string(val))
|
||||
assert.NoError(t, err)
|
||||
|
||||
// wait for check goroutine received 2 events
|
||||
<-c
|
||||
exist := node.flowgraphManager.HasFlowgraph(ch)
|
||||
assert.False(t, exist)
|
||||
|
||||
err = kv.RemoveWithPrefix(fmt.Sprintf("%s/%d", Params.CommonCfg.DataCoordWatchSubPath.GetValue(), paramtable.GetNodeID()))
|
||||
assert.NoError(t, err)
|
||||
// TODO there is not way to sync Release done, use sleep for now
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
exist = node.flowgraphManager.HasFlowgraph(ch)
|
||||
assert.False(t, exist)
|
||||
})
|
||||
|
||||
t.Run("handle watch info failed", func(t *testing.T) {
|
||||
e := &event{
|
||||
eventType: putEventType,
|
||||
}
|
||||
|
||||
node.handleWatchInfo(e, "test1", []byte{23})
|
||||
|
||||
exist := node.flowgraphManager.HasFlowgraph("test1")
|
||||
assert.False(t, exist)
|
||||
|
||||
info := datapb.ChannelWatchInfo{
|
||||
Vchan: nil,
|
||||
State: datapb.ChannelWatchState_Uncomplete,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
bs, err := proto.Marshal(&info)
|
||||
assert.NoError(t, err)
|
||||
node.handleWatchInfo(e, "test2", bs)
|
||||
|
||||
exist = node.flowgraphManager.HasFlowgraph("test2")
|
||||
assert.False(t, exist)
|
||||
|
||||
chPut := make(chan struct{}, 1)
|
||||
chDel := make(chan struct{}, 1)
|
||||
|
||||
ch := fmt.Sprintf("datanode-etcd-test-by-dev-rootcoord-dml-channel_%d", rand.Int31())
|
||||
m := newChannelEventManager(
|
||||
func(info *datapb.ChannelWatchInfo, version int64) error {
|
||||
r := node.handlePutEvent(info, version)
|
||||
chPut <- struct{}{}
|
||||
return r
|
||||
},
|
||||
func(vChan string) {
|
||||
node.handleDeleteEvent(vChan)
|
||||
chDel <- struct{}{}
|
||||
}, time.Millisecond*100,
|
||||
)
|
||||
|
||||
node.eventManager.GetOrInsert(ch, m)
|
||||
defer node.eventManager.Remove(ch)
|
||||
|
||||
info = datapb.ChannelWatchInfo{
|
||||
Vchan: &datapb.VchannelInfo{ChannelName: ch},
|
||||
State: datapb.ChannelWatchState_Uncomplete,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
bs, err = proto.Marshal(&info)
|
||||
assert.NoError(t, err)
|
||||
|
||||
msFactory := node.factory
|
||||
defer func() { node.factory = msFactory }()
|
||||
|
||||
// todo review the UT logic
|
||||
// As we remove timetick channel logic, flow_graph_insert_buffer_node no longer depend on MessageStreamFactory
|
||||
// so data_sync_service can be created. this assert becomes true
|
||||
node.factory = &FailMessageStreamFactory{}
|
||||
node.handleWatchInfo(e, ch, bs)
|
||||
<-chPut
|
||||
exist = node.flowgraphManager.HasFlowgraph(ch)
|
||||
assert.True(t, exist)
|
||||
})
|
||||
|
||||
t.Run("handle watchinfo out of date", func(t *testing.T) {
|
||||
chPut := make(chan struct{}, 1)
|
||||
chDel := make(chan struct{}, 1)
|
||||
// inject eventManager
|
||||
ch := fmt.Sprintf("datanode-etcd-test-by-dev-rootcoord-dml-channel_%d", rand.Int31())
|
||||
m := newChannelEventManager(
|
||||
func(info *datapb.ChannelWatchInfo, version int64) error {
|
||||
r := node.handlePutEvent(info, version)
|
||||
chPut <- struct{}{}
|
||||
return r
|
||||
},
|
||||
func(vChan string) {
|
||||
node.handleDeleteEvent(vChan)
|
||||
chDel <- struct{}{}
|
||||
}, time.Millisecond*100,
|
||||
)
|
||||
node.eventManager.GetOrInsert(ch, m)
|
||||
defer node.eventManager.Remove(ch)
|
||||
e := &event{
|
||||
eventType: putEventType,
|
||||
version: 10000,
|
||||
}
|
||||
|
||||
info := datapb.ChannelWatchInfo{
|
||||
Vchan: &datapb.VchannelInfo{ChannelName: ch},
|
||||
State: datapb.ChannelWatchState_Uncomplete,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
bs, err := proto.Marshal(&info)
|
||||
assert.NoError(t, err)
|
||||
|
||||
node.handleWatchInfo(e, ch, bs)
|
||||
<-chPut
|
||||
exist := node.flowgraphManager.HasFlowgraph("test3")
|
||||
assert.False(t, exist)
|
||||
})
|
||||
|
||||
t.Run("handle watchinfo compatibility", func(t *testing.T) {
|
||||
info := datapb.ChannelWatchInfo{
|
||||
Vchan: &datapb.VchannelInfo{
|
||||
CollectionID: 1,
|
||||
ChannelName: "delta-channel1",
|
||||
UnflushedSegments: []*datapb.SegmentInfo{{ID: 1}},
|
||||
FlushedSegments: []*datapb.SegmentInfo{{ID: 2}},
|
||||
DroppedSegments: []*datapb.SegmentInfo{{ID: 3}},
|
||||
UnflushedSegmentIds: []int64{1},
|
||||
},
|
||||
State: datapb.ChannelWatchState_Uncomplete,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
bs, err := proto.Marshal(&info)
|
||||
assert.NoError(t, err)
|
||||
|
||||
newWatchInfo, err := parsePutEventData(bs)
|
||||
assert.NoError(t, err)
|
||||
|
||||
assert.Equal(t, []*datapb.SegmentInfo{}, newWatchInfo.GetVchan().GetUnflushedSegments())
|
||||
assert.Equal(t, []*datapb.SegmentInfo{}, newWatchInfo.GetVchan().GetFlushedSegments())
|
||||
assert.Equal(t, []*datapb.SegmentInfo{}, newWatchInfo.GetVchan().GetDroppedSegments())
|
||||
assert.NotEmpty(t, newWatchInfo.GetVchan().GetUnflushedSegmentIds())
|
||||
assert.NotEmpty(t, newWatchInfo.GetVchan().GetFlushedSegmentIds())
|
||||
assert.NotEmpty(t, newWatchInfo.GetVchan().GetDroppedSegmentIds())
|
||||
})
|
||||
}
|
||||
|
||||
func TestChannelEventManager(t *testing.T) {
|
||||
t.Run("normal case", func(t *testing.T) {
|
||||
ch := make(chan struct{}, 1)
|
||||
ran := false
|
||||
em := newChannelEventManager(func(info *datapb.ChannelWatchInfo, version int64) error {
|
||||
ran = true
|
||||
ch <- struct{}{}
|
||||
return nil
|
||||
}, func(name string) {}, time.Millisecond*10)
|
||||
|
||||
em.Run()
|
||||
em.handleEvent(event{
|
||||
eventType: putEventType,
|
||||
vChanName: "",
|
||||
version: 0,
|
||||
info: &datapb.ChannelWatchInfo{},
|
||||
})
|
||||
<-ch
|
||||
assert.True(t, ran)
|
||||
})
|
||||
|
||||
t.Run("close behavior", func(t *testing.T) {
|
||||
ch := make(chan struct{}, 1)
|
||||
em := newChannelEventManager(func(info *datapb.ChannelWatchInfo, version int64) error {
|
||||
return errors.New("mocked error")
|
||||
}, func(name string) {}, time.Millisecond*10)
|
||||
|
||||
go func() {
|
||||
evt := event{
|
||||
eventType: putEventType,
|
||||
vChanName: "",
|
||||
version: 0,
|
||||
info: &datapb.ChannelWatchInfo{},
|
||||
}
|
||||
em.handleEvent(evt)
|
||||
ch <- struct{}{}
|
||||
}()
|
||||
|
||||
select {
|
||||
case <-ch:
|
||||
case <-time.After(time.Second):
|
||||
t.FailNow()
|
||||
}
|
||||
close(em.eventChan)
|
||||
|
||||
assert.NotPanics(t, func() {
|
||||
em.Close()
|
||||
em.Close()
|
||||
})
|
||||
})
|
||||
|
||||
t.Run("cancel by delete event", func(t *testing.T) {
|
||||
ch := make(chan struct{}, 1)
|
||||
ran := false
|
||||
em := newChannelEventManager(
|
||||
func(info *datapb.ChannelWatchInfo, version int64) error {
|
||||
return errors.New("mocked error")
|
||||
},
|
||||
func(name string) {
|
||||
ran = true
|
||||
ch <- struct{}{}
|
||||
},
|
||||
time.Millisecond*10,
|
||||
)
|
||||
em.Run()
|
||||
em.handleEvent(event{
|
||||
eventType: putEventType,
|
||||
vChanName: "",
|
||||
version: 0,
|
||||
info: &datapb.ChannelWatchInfo{},
|
||||
})
|
||||
em.handleEvent(event{
|
||||
eventType: deleteEventType,
|
||||
vChanName: "",
|
||||
version: 0,
|
||||
info: &datapb.ChannelWatchInfo{},
|
||||
})
|
||||
<-ch
|
||||
assert.True(t, ran)
|
||||
})
|
||||
|
||||
t.Run("overwrite put event", func(t *testing.T) {
|
||||
ch := make(chan struct{}, 1)
|
||||
ran := false
|
||||
em := newChannelEventManager(
|
||||
func(info *datapb.ChannelWatchInfo, version int64) error {
|
||||
if version > 0 {
|
||||
ran = true
|
||||
ch <- struct{}{}
|
||||
return nil
|
||||
}
|
||||
return errors.New("mocked error")
|
||||
},
|
||||
func(name string) {},
|
||||
time.Millisecond*10)
|
||||
em.Run()
|
||||
em.handleEvent(event{
|
||||
eventType: putEventType,
|
||||
vChanName: "",
|
||||
version: 0,
|
||||
info: &datapb.ChannelWatchInfo{
|
||||
State: datapb.ChannelWatchState_ToWatch,
|
||||
},
|
||||
})
|
||||
em.handleEvent(event{
|
||||
eventType: putEventType,
|
||||
vChanName: "",
|
||||
version: 1,
|
||||
info: &datapb.ChannelWatchInfo{
|
||||
State: datapb.ChannelWatchState_ToWatch,
|
||||
},
|
||||
})
|
||||
<-ch
|
||||
assert.True(t, ran)
|
||||
})
|
||||
}
|
||||
|
||||
func parseWatchInfo(key string, data []byte) (*datapb.ChannelWatchInfo, error) {
|
||||
watchInfo := datapb.ChannelWatchInfo{}
|
||||
if err := proto.Unmarshal(data, &watchInfo); err != nil {
|
||||
return nil, fmt.Errorf("invalid event data: fail to parse ChannelWatchInfo, key: %s, err: %v", key, err)
|
||||
}
|
||||
|
||||
if watchInfo.Vchan == nil {
|
||||
return nil, fmt.Errorf("invalid event: ChannelWatchInfo with nil VChannelInfo, key: %s", key)
|
||||
}
|
||||
reviseVChannelInfo(watchInfo.GetVchan())
|
||||
|
||||
return &watchInfo, nil
|
||||
}
|
||||
|
||||
func TestEventTickler(t *testing.T) {
|
||||
channelName := "test-channel"
|
||||
etcdPrefix := "test_path"
|
||||
|
||||
kv, err := newTestEtcdKV()
|
||||
assert.NoError(t, err)
|
||||
kv.RemoveWithPrefix(etcdPrefix)
|
||||
defer kv.RemoveWithPrefix(etcdPrefix)
|
||||
|
||||
tickler := newEtcdTickler(0, path.Join(etcdPrefix, channelName), &datapb.ChannelWatchInfo{
|
||||
Vchan: &datapb.VchannelInfo{
|
||||
ChannelName: channelName,
|
||||
},
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, kv, 100*time.Millisecond)
|
||||
defer tickler.stop()
|
||||
endCh := make(chan struct{}, 1)
|
||||
go func() {
|
||||
watchCh := kv.WatchWithPrefix(etcdPrefix)
|
||||
for {
|
||||
event, ok := <-watchCh
|
||||
assert.True(t, ok)
|
||||
for _, evt := range event.Events {
|
||||
key := string(evt.Kv.Key)
|
||||
watchInfo, err := parseWatchInfo(key, evt.Kv.Value)
|
||||
assert.NoError(t, err)
|
||||
if watchInfo.GetVchan().GetChannelName() == channelName {
|
||||
assert.Equal(t, int32(1), watchInfo.Progress)
|
||||
endCh <- struct{}{}
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
tickler.inc()
|
||||
tickler.watch()
|
||||
assert.Eventually(t, func() bool {
|
||||
select {
|
||||
case <-endCh:
|
||||
return true
|
||||
default:
|
||||
return false
|
||||
}
|
||||
}, 4*time.Second, 100*time.Millisecond)
|
||||
}
|
@ -1,146 +0,0 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/require"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/broker"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/util/etcd"
|
||||
)
|
||||
|
||||
func TestFlowGraphManager(t *testing.T) {
|
||||
ctx, cancel := context.WithCancel(context.TODO())
|
||||
defer cancel()
|
||||
|
||||
etcdCli, err := etcd.GetEtcdClient(
|
||||
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
|
||||
Params.EtcdCfg.EtcdUseSSL.GetAsBool(),
|
||||
Params.EtcdCfg.Endpoints.GetAsStrings(),
|
||||
Params.EtcdCfg.EtcdTLSCert.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSKey.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSCACert.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSMinVersion.GetValue())
|
||||
assert.NoError(t, err)
|
||||
defer etcdCli.Close()
|
||||
|
||||
node := newIDLEDataNodeMock(ctx, schemapb.DataType_Int64)
|
||||
defer node.Stop()
|
||||
node.SetEtcdClient(etcdCli)
|
||||
err = node.Init()
|
||||
require.Nil(t, err)
|
||||
|
||||
broker := broker.NewMockBroker(t)
|
||||
broker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
broker.EXPECT().SaveBinlogPaths(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
broker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).Return([]*datapb.SegmentInfo{}, nil).Maybe()
|
||||
broker.EXPECT().DropVirtualChannel(mock.Anything, mock.Anything).Return(nil, nil).Maybe()
|
||||
broker.EXPECT().UpdateChannelCheckpoint(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
|
||||
node.broker = broker
|
||||
|
||||
fm := newFlowgraphManager()
|
||||
defer func() {
|
||||
fm.ClearFlowgraphs()
|
||||
}()
|
||||
|
||||
t.Run("Test addAndStart", func(t *testing.T) {
|
||||
vchanName := "by-dev-rootcoord-dml-test-flowgraphmanager-addAndStart"
|
||||
vchan := &datapb.VchannelInfo{
|
||||
CollectionID: 1,
|
||||
ChannelName: vchanName,
|
||||
}
|
||||
require.False(t, fm.HasFlowgraph(vchanName))
|
||||
|
||||
err := fm.AddandStartWithEtcdTickler(node, vchan, &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, genTestTickler())
|
||||
assert.NoError(t, err)
|
||||
assert.True(t, fm.HasFlowgraph(vchanName))
|
||||
|
||||
fm.ClearFlowgraphs()
|
||||
})
|
||||
|
||||
t.Run("Test Release", func(t *testing.T) {
|
||||
vchanName := "by-dev-rootcoord-dml-test-flowgraphmanager-Release"
|
||||
vchan := &datapb.VchannelInfo{
|
||||
CollectionID: 1,
|
||||
ChannelName: vchanName,
|
||||
}
|
||||
require.False(t, fm.HasFlowgraph(vchanName))
|
||||
|
||||
err := fm.AddandStartWithEtcdTickler(node, vchan, &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, genTestTickler())
|
||||
assert.NoError(t, err)
|
||||
assert.True(t, fm.HasFlowgraph(vchanName))
|
||||
|
||||
fm.RemoveFlowgraph(vchanName)
|
||||
|
||||
assert.False(t, fm.HasFlowgraph(vchanName))
|
||||
fm.ClearFlowgraphs()
|
||||
})
|
||||
|
||||
t.Run("Test getFlowgraphService", func(t *testing.T) {
|
||||
fg, ok := fm.GetFlowgraphService("channel-not-exist")
|
||||
assert.False(t, ok)
|
||||
assert.Nil(t, fg)
|
||||
})
|
||||
}
|
@ -1,79 +0,0 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"go.uber.org/atomic"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
type flushTaskCounter struct {
|
||||
inner *typeutil.ConcurrentMap[string, *atomic.Int32] // channel -> counter
|
||||
}
|
||||
|
||||
func (c *flushTaskCounter) getOrZero(channel string) int32 {
|
||||
counter, exist := c.inner.Get(channel)
|
||||
if !exist {
|
||||
return 0
|
||||
}
|
||||
return counter.Load()
|
||||
}
|
||||
|
||||
func (c *flushTaskCounter) increaseImpl(channel string, delta int32) {
|
||||
counter, _ := c.inner.GetOrInsert(channel, atomic.NewInt32(0))
|
||||
counter.Add(delta)
|
||||
}
|
||||
|
||||
func (c *flushTaskCounter) increase(channel string) {
|
||||
c.increaseImpl(channel, 1)
|
||||
}
|
||||
|
||||
func (c *flushTaskCounter) decrease(channel string) {
|
||||
c.increaseImpl(channel, -1)
|
||||
}
|
||||
|
||||
func (c *flushTaskCounter) close() {
|
||||
allChannels := make([]string, 0, c.inner.Len())
|
||||
c.inner.Range(func(channel string, _ *atomic.Int32) bool {
|
||||
allChannels = append(allChannels, channel)
|
||||
return false
|
||||
})
|
||||
for _, channel := range allChannels {
|
||||
c.inner.Remove(channel)
|
||||
}
|
||||
}
|
||||
|
||||
func newFlushTaskCounter() *flushTaskCounter {
|
||||
return &flushTaskCounter{
|
||||
inner: typeutil.NewConcurrentMap[string, *atomic.Int32](),
|
||||
}
|
||||
}
|
||||
|
||||
var (
|
||||
globalFlushTaskCounter *flushTaskCounter
|
||||
flushTaskCounterOnce sync.Once
|
||||
)
|
||||
|
||||
func getOrCreateFlushTaskCounter() *flushTaskCounter {
|
||||
flushTaskCounterOnce.Do(func() {
|
||||
globalFlushTaskCounter = newFlushTaskCounter()
|
||||
})
|
||||
return globalFlushTaskCounter
|
||||
}
|
@ -1,44 +0,0 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func Test_flushTaskCounter_getOrZero(t *testing.T) {
|
||||
c := newFlushTaskCounter()
|
||||
defer c.close()
|
||||
|
||||
assert.Zero(t, c.getOrZero("non-exist"))
|
||||
|
||||
n := 10
|
||||
channel := "channel"
|
||||
assert.Zero(t, c.getOrZero(channel))
|
||||
|
||||
for i := 0; i < n; i++ {
|
||||
c.increase(channel)
|
||||
}
|
||||
assert.Equal(t, int32(n), c.getOrZero(channel))
|
||||
|
||||
for i := 0; i < n; i++ {
|
||||
c.decrease(channel)
|
||||
}
|
||||
assert.Zero(t, c.getOrZero(channel))
|
||||
}
|
@ -20,6 +20,7 @@ import (
|
||||
"context"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/pkg/util/hardware"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
|
||||
@ -33,7 +34,7 @@ func (node *DataNode) getQuotaMetrics() (*metricsinfo.DataNodeQuotaMetrics, erro
|
||||
var err error
|
||||
rms := make([]metricsinfo.RateMetric, 0)
|
||||
getRateMetric := func(label metricsinfo.RateMetricLabel) {
|
||||
rate, err2 := rateCol.Rate(label, ratelimitutil.DefaultAvgDuration)
|
||||
rate, err2 := util.RateCol.Rate(label, ratelimitutil.DefaultAvgDuration)
|
||||
if err2 != nil {
|
||||
err = err2
|
||||
return
|
||||
@ -49,7 +50,7 @@ func (node *DataNode) getQuotaMetrics() (*metricsinfo.DataNodeQuotaMetrics, erro
|
||||
return nil, err
|
||||
}
|
||||
|
||||
minFGChannel, minFGTt := rateCol.getMinFlowGraphTt()
|
||||
minFGChannel, minFGTt := util.RateCol.GetMinFlowGraphTt()
|
||||
return &metricsinfo.DataNodeQuotaMetrics{
|
||||
Hms: metricsinfo.HardwareMetrics{},
|
||||
Rms: rms,
|
||||
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"context"
|
||||
@ -23,7 +23,6 @@ import (
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/broker"
|
||||
"github.com/milvus-io/milvus/internal/datanode/compaction"
|
||||
"github.com/milvus-io/milvus/internal/datanode/io"
|
||||
@ -45,33 +44,27 @@ import (
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
// dataSyncService controls a flowgraph for a specific collection
|
||||
type dataSyncService struct {
|
||||
// DataSyncService controls a flowgraph for a specific collection
|
||||
type DataSyncService struct {
|
||||
ctx context.Context
|
||||
cancelFn context.CancelFunc
|
||||
metacache metacache.MetaCache
|
||||
opID int64
|
||||
collectionID UniqueID // collection id of vchan for which this data sync service serves
|
||||
collectionID util.UniqueID // collection id of vchan for which this data sync service serves
|
||||
vchannelName string
|
||||
|
||||
// TODO: should be equal to paramtable.GetNodeID(), but intergrationtest has 1 paramtable for a minicluster, the NodeID
|
||||
// varies, will cause savebinglogpath check fail. So we pass ServerID into dataSyncService to aviod it failure.
|
||||
serverID UniqueID
|
||||
// varies, will cause savebinglogpath check fail. So we pass ServerID into DataSyncService to aviod it failure.
|
||||
serverID util.UniqueID
|
||||
|
||||
fg *flowgraph.TimeTickedFlowGraph // internal flowgraph processes insert/delta messages
|
||||
|
||||
broker broker.Broker
|
||||
syncMgr syncmgr.SyncManager
|
||||
|
||||
flushCh chan flushMsg
|
||||
resendTTCh chan resendTTMsg // chan to ask for resending DataNode time tick message.
|
||||
timetickSender *timeTickSender // reference to timeTickSender
|
||||
compactor compaction.Executor // reference to compaction executor
|
||||
flushingSegCache *Cache // a guarding cache stores currently flushing segment ids
|
||||
timetickSender *util.TimeTickSender // reference to TimeTickSender
|
||||
compactor compaction.Executor // reference to compaction executor
|
||||
|
||||
clearSignal chan<- string // signal channel to notify flowgraph close for collection/partition drop msg consumed
|
||||
idAllocator allocator.Allocator // id/timestamp allocator
|
||||
msFactory msgstream.Factory
|
||||
dispClient msgdispatcher.Client
|
||||
chunkManager storage.ChunkManager
|
||||
|
||||
@ -80,15 +73,14 @@ type dataSyncService struct {
|
||||
|
||||
type nodeConfig struct {
|
||||
msFactory msgstream.Factory // msgStream factory
|
||||
collectionID UniqueID
|
||||
collectionID util.UniqueID
|
||||
vChannelName string
|
||||
metacache metacache.MetaCache
|
||||
allocator allocator.Allocator
|
||||
serverID UniqueID
|
||||
serverID util.UniqueID
|
||||
}
|
||||
|
||||
// start the flow graph in dataSyncService
|
||||
func (dsService *dataSyncService) start() {
|
||||
// Start the flow graph in dataSyncService
|
||||
func (dsService *DataSyncService) Start() {
|
||||
if dsService.fg != nil {
|
||||
log.Info("dataSyncService starting flow graph", zap.Int64("collectionID", dsService.collectionID),
|
||||
zap.String("vChanName", dsService.vchannelName))
|
||||
@ -99,7 +91,7 @@ func (dsService *dataSyncService) start() {
|
||||
}
|
||||
}
|
||||
|
||||
func (dsService *dataSyncService) GracefullyClose() {
|
||||
func (dsService *DataSyncService) GracefullyClose() {
|
||||
if dsService.fg != nil {
|
||||
log.Info("dataSyncService gracefully closing flowgraph")
|
||||
dsService.fg.SetCloseMethod(flowgraph.CloseGracefully)
|
||||
@ -107,7 +99,11 @@ func (dsService *dataSyncService) GracefullyClose() {
|
||||
}
|
||||
}
|
||||
|
||||
func (dsService *dataSyncService) close() {
|
||||
func (dsService *DataSyncService) GetOpID() int64 {
|
||||
return dsService.opID
|
||||
}
|
||||
|
||||
func (dsService *DataSyncService) close() {
|
||||
dsService.stopOnce.Do(func() {
|
||||
log := log.Ctx(dsService.ctx).With(
|
||||
zap.Int64("collectionID", dsService.collectionID),
|
||||
@ -130,19 +126,16 @@ func (dsService *dataSyncService) close() {
|
||||
})
|
||||
}
|
||||
|
||||
func getMetaCacheWithTickler(initCtx context.Context, node *DataNode, info *datapb.ChannelWatchInfo, tickler *tickler, unflushed, flushed []*datapb.SegmentInfo, storageV2Cache *metacache.StorageV2Cache) (metacache.MetaCache, error) {
|
||||
tickler.setTotal(int32(len(unflushed) + len(flushed)))
|
||||
return initMetaCache(initCtx, storageV2Cache, node.chunkManager, info, tickler, unflushed, flushed)
|
||||
func (dsService *DataSyncService) GetMetaCache() metacache.MetaCache {
|
||||
return dsService.metacache
|
||||
}
|
||||
|
||||
func getMetaCacheWithEtcdTickler(initCtx context.Context, node *DataNode, info *datapb.ChannelWatchInfo, tickler *etcdTickler, unflushed, flushed []*datapb.SegmentInfo, storageV2Cache *metacache.StorageV2Cache) (metacache.MetaCache, error) {
|
||||
tickler.watch()
|
||||
defer tickler.stop()
|
||||
|
||||
return initMetaCache(initCtx, storageV2Cache, node.chunkManager, info, tickler, unflushed, flushed)
|
||||
func getMetaCacheWithTickler(initCtx context.Context, params *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler, unflushed, flushed []*datapb.SegmentInfo, storageV2Cache *metacache.StorageV2Cache) (metacache.MetaCache, error) {
|
||||
tickler.SetTotal(int32(len(unflushed) + len(flushed)))
|
||||
return initMetaCache(initCtx, storageV2Cache, params.ChunkManager, info, tickler, unflushed, flushed)
|
||||
}
|
||||
|
||||
func initMetaCache(initCtx context.Context, storageV2Cache *metacache.StorageV2Cache, chunkManager storage.ChunkManager, info *datapb.ChannelWatchInfo, tickler interface{ inc() }, unflushed, flushed []*datapb.SegmentInfo) (metacache.MetaCache, error) {
|
||||
func initMetaCache(initCtx context.Context, storageV2Cache *metacache.StorageV2Cache, chunkManager storage.ChunkManager, info *datapb.ChannelWatchInfo, tickler interface{ Inc() }, unflushed, flushed []*datapb.SegmentInfo) (metacache.MetaCache, error) {
|
||||
// tickler will update addSegment progress to watchInfo
|
||||
futures := make([]*conc.Future[any], 0, len(unflushed)+len(flushed))
|
||||
segmentPks := typeutil.NewConcurrentMap[int64, []*storage.PkStatistics]()
|
||||
@ -161,15 +154,15 @@ func initMetaCache(initCtx context.Context, storageV2Cache *metacache.StorageV2C
|
||||
var stats []*storage.PkStatistics
|
||||
var err error
|
||||
if params.Params.CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
stats, err = util.LoadStatsV2(storageV2Cache, segment, info.GetSchema())
|
||||
stats, err = compaction.LoadStatsV2(storageV2Cache, segment, info.GetSchema())
|
||||
} else {
|
||||
stats, err = util.LoadStats(initCtx, chunkManager, info.GetSchema(), segment.GetID(), segment.GetStatslogs())
|
||||
stats, err = compaction.LoadStats(initCtx, chunkManager, info.GetSchema(), segment.GetID(), segment.GetStatslogs())
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
segmentPks.Insert(segment.GetID(), stats)
|
||||
tickler.inc()
|
||||
tickler.Inc()
|
||||
|
||||
return struct{}{}, nil
|
||||
})
|
||||
@ -198,88 +191,77 @@ func initMetaCache(initCtx context.Context, storageV2Cache *metacache.StorageV2C
|
||||
return metacache, nil
|
||||
}
|
||||
|
||||
func getServiceWithChannel(initCtx context.Context, node *DataNode, info *datapb.ChannelWatchInfo, metacache metacache.MetaCache, storageV2Cache *metacache.StorageV2Cache, unflushed, flushed []*datapb.SegmentInfo) (*dataSyncService, error) {
|
||||
func getServiceWithChannel(initCtx context.Context, params *util.PipelineParams, info *datapb.ChannelWatchInfo, metacache metacache.MetaCache, storageV2Cache *metacache.StorageV2Cache, unflushed, flushed []*datapb.SegmentInfo) (*DataSyncService, error) {
|
||||
var (
|
||||
channelName = info.GetVchan().GetChannelName()
|
||||
collectionID = info.GetVchan().GetCollectionID()
|
||||
)
|
||||
|
||||
config := &nodeConfig{
|
||||
msFactory: node.factory,
|
||||
allocator: node.allocator,
|
||||
|
||||
msFactory: params.MsgStreamFactory,
|
||||
collectionID: collectionID,
|
||||
vChannelName: channelName,
|
||||
metacache: metacache,
|
||||
serverID: node.session.ServerID,
|
||||
serverID: params.Session.ServerID,
|
||||
}
|
||||
|
||||
var (
|
||||
flushCh = make(chan flushMsg, 100)
|
||||
resendTTCh = make(chan resendTTMsg, 100)
|
||||
)
|
||||
|
||||
err := node.writeBufferManager.Register(channelName, metacache, storageV2Cache, writebuffer.WithMetaWriter(syncmgr.BrokerMetaWriter(node.broker, config.serverID)), writebuffer.WithIDAllocator(node.allocator))
|
||||
err := params.WriteBufferManager.Register(channelName, metacache, storageV2Cache,
|
||||
writebuffer.WithMetaWriter(syncmgr.BrokerMetaWriter(params.Broker, config.serverID)),
|
||||
writebuffer.WithIDAllocator(params.Allocator))
|
||||
if err != nil {
|
||||
log.Warn("failed to register channel buffer", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
defer func() {
|
||||
if err != nil {
|
||||
defer node.writeBufferManager.RemoveChannel(channelName)
|
||||
defer params.WriteBufferManager.RemoveChannel(channelName)
|
||||
}
|
||||
}()
|
||||
|
||||
ctx, cancel := context.WithCancel(node.ctx)
|
||||
ds := &dataSyncService{
|
||||
ctx: ctx,
|
||||
cancelFn: cancel,
|
||||
flushCh: flushCh,
|
||||
resendTTCh: resendTTCh,
|
||||
opID: info.GetOpID(),
|
||||
ctx, cancel := context.WithCancel(params.Ctx)
|
||||
ds := &DataSyncService{
|
||||
ctx: ctx,
|
||||
cancelFn: cancel,
|
||||
opID: info.GetOpID(),
|
||||
|
||||
dispClient: node.dispClient,
|
||||
msFactory: node.factory,
|
||||
broker: node.broker,
|
||||
dispClient: params.DispClient,
|
||||
broker: params.Broker,
|
||||
|
||||
idAllocator: config.allocator,
|
||||
metacache: config.metacache,
|
||||
collectionID: config.collectionID,
|
||||
vchannelName: config.vChannelName,
|
||||
serverID: config.serverID,
|
||||
|
||||
flushingSegCache: node.segmentCache,
|
||||
clearSignal: node.clearSignal,
|
||||
chunkManager: node.chunkManager,
|
||||
compactor: node.compactionExecutor,
|
||||
timetickSender: node.timeTickSender,
|
||||
syncMgr: node.syncMgr,
|
||||
chunkManager: params.ChunkManager,
|
||||
compactor: params.CompactionExecutor,
|
||||
timetickSender: params.TimeTickSender,
|
||||
syncMgr: params.SyncMgr,
|
||||
|
||||
fg: nil,
|
||||
}
|
||||
|
||||
// init flowgraph
|
||||
fg := flowgraph.NewTimeTickedFlowGraph(node.ctx)
|
||||
dmStreamNode, err := newDmInputNode(initCtx, node.dispClient, info.GetVchan().GetSeekPosition(), config)
|
||||
fg := flowgraph.NewTimeTickedFlowGraph(params.Ctx)
|
||||
dmStreamNode, err := newDmInputNode(initCtx, params.DispClient, info.GetVchan().GetSeekPosition(), config)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
ddNode, err := newDDNode(
|
||||
node.ctx,
|
||||
params.Ctx,
|
||||
collectionID,
|
||||
channelName,
|
||||
info.GetVchan().GetDroppedSegmentIds(),
|
||||
flushed,
|
||||
unflushed,
|
||||
node.compactionExecutor,
|
||||
params.CompactionExecutor,
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var updater statsUpdater
|
||||
if Params.DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() {
|
||||
if paramtable.Get().DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() {
|
||||
updater = ds.timetickSender
|
||||
} else {
|
||||
m, err := config.msFactory.NewMsgStream(ctx)
|
||||
@ -287,18 +269,18 @@ func getServiceWithChannel(initCtx context.Context, node *DataNode, info *datapb
|
||||
return nil, err
|
||||
}
|
||||
|
||||
m.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()})
|
||||
m.AsProducer([]string{paramtable.Get().CommonCfg.DataCoordTimeTick.GetValue()})
|
||||
metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(config.serverID)).Inc()
|
||||
log.Info("datanode AsProducer", zap.String("TimeTickChannelName", Params.CommonCfg.DataCoordTimeTick.GetValue()))
|
||||
log.Info("datanode AsProducer", zap.String("TimeTickChannelName", paramtable.Get().CommonCfg.DataCoordTimeTick.GetValue()))
|
||||
|
||||
m.EnableProduce(true)
|
||||
|
||||
updater = newMqStatsUpdater(config, m)
|
||||
}
|
||||
|
||||
writeNode := newWriteNode(node.ctx, node.writeBufferManager, updater, config)
|
||||
writeNode := newWriteNode(params.Ctx, params.WriteBufferManager, updater, config)
|
||||
|
||||
ttNode, err := newTTNode(config, node.writeBufferManager, node.channelCheckpointUpdater)
|
||||
ttNode, err := newTTNode(config, params.WriteBufferManager, params.CheckpointUpdater)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -311,16 +293,16 @@ func getServiceWithChannel(initCtx context.Context, node *DataNode, info *datapb
|
||||
return ds, nil
|
||||
}
|
||||
|
||||
// newServiceWithEtcdTickler gets a dataSyncService, but flowgraphs are not running
|
||||
// NewDataSyncService gets a dataSyncService, but flowgraphs are not running
|
||||
// initCtx is used to init the dataSyncService only, if initCtx.Canceled or initCtx.Timeout
|
||||
// newServiceWithEtcdTickler stops and returns the initCtx.Err()
|
||||
func newServiceWithEtcdTickler(initCtx context.Context, node *DataNode, info *datapb.ChannelWatchInfo, tickler *etcdTickler) (*dataSyncService, error) {
|
||||
// NewDataSyncService stops and returns the initCtx.Err()
|
||||
func NewDataSyncService(initCtx context.Context, pipelineParams *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler) (*DataSyncService, error) {
|
||||
// recover segment checkpoints
|
||||
unflushedSegmentInfos, err := node.broker.GetSegmentInfo(initCtx, info.GetVchan().GetUnflushedSegmentIds())
|
||||
unflushedSegmentInfos, err := pipelineParams.Broker.GetSegmentInfo(initCtx, info.GetVchan().GetUnflushedSegmentIds())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
flushedSegmentInfos, err := node.broker.GetSegmentInfo(initCtx, info.GetVchan().GetFlushedSegmentIds())
|
||||
flushedSegmentInfos, err := pipelineParams.Broker.GetSegmentInfo(initCtx, info.GetVchan().GetFlushedSegmentIds())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -332,42 +314,16 @@ func newServiceWithEtcdTickler(initCtx context.Context, node *DataNode, info *da
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
// init channel meta
|
||||
metaCache, err := getMetaCacheWithEtcdTickler(initCtx, node, info, tickler, unflushedSegmentInfos, flushedSegmentInfos, storageCache)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return getServiceWithChannel(initCtx, node, info, metaCache, storageCache, unflushedSegmentInfos, flushedSegmentInfos)
|
||||
}
|
||||
|
||||
// newDataSyncService gets a dataSyncService, but flowgraphs are not running
|
||||
// initCtx is used to init the dataSyncService only, if initCtx.Canceled or initCtx.Timeout
|
||||
// newDataSyncService stops and returns the initCtx.Err()
|
||||
// NOTE: compactiable for event manager
|
||||
func newDataSyncService(initCtx context.Context, node *DataNode, info *datapb.ChannelWatchInfo, tickler *tickler) (*dataSyncService, error) {
|
||||
// recover segment checkpoints
|
||||
unflushedSegmentInfos, err := node.broker.GetSegmentInfo(initCtx, info.GetVchan().GetUnflushedSegmentIds())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
flushedSegmentInfos, err := node.broker.GetSegmentInfo(initCtx, info.GetVchan().GetFlushedSegmentIds())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var storageCache *metacache.StorageV2Cache
|
||||
if params.Params.CommonCfg.EnableStorageV2.GetAsBool() {
|
||||
storageCache, err = metacache.NewStorageV2Cache(info.Schema)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
// init metaCache meta
|
||||
metaCache, err := getMetaCacheWithTickler(initCtx, node, info, tickler, unflushedSegmentInfos, flushedSegmentInfos, storageCache)
|
||||
metaCache, err := getMetaCacheWithTickler(initCtx, pipelineParams, info, tickler, unflushedSegmentInfos, flushedSegmentInfos, storageCache)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return getServiceWithChannel(initCtx, node, info, metaCache, storageCache, unflushedSegmentInfos, flushedSegmentInfos)
|
||||
return getServiceWithChannel(initCtx, pipelineParams, info, metaCache, storageCache, unflushedSegmentInfos, flushedSegmentInfos)
|
||||
}
|
||||
|
||||
func NewDataSyncServiceWithMetaCache(metaCache metacache.MetaCache) *DataSyncService {
|
||||
return &DataSyncService{metacache: metaCache}
|
||||
}
|
@ -14,12 +14,10 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"math"
|
||||
"math/rand"
|
||||
@ -29,7 +27,6 @@ import (
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/suite"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
@ -37,6 +34,8 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/broker"
|
||||
"github.com/milvus-io/milvus/internal/datanode/metacache"
|
||||
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/datanode/writebuffer"
|
||||
"github.com/milvus-io/milvus/internal/mocks"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
@ -44,7 +43,6 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
@ -54,10 +52,6 @@ import (
|
||||
|
||||
var dataSyncServiceTestDir = "/tmp/milvus_test/data_sync_service"
|
||||
|
||||
func init() {
|
||||
paramtable.Init()
|
||||
}
|
||||
|
||||
func getWatchInfo(info *testInfo) *datapb.ChannelWatchInfo {
|
||||
return &datapb.ChannelWatchInfo{
|
||||
Vchan: getVchanInfo(info),
|
||||
@ -131,16 +125,16 @@ type testInfo struct {
|
||||
channelNil bool
|
||||
inMsgFactory dependency.Factory
|
||||
|
||||
collID UniqueID
|
||||
collID util.UniqueID
|
||||
chanName string
|
||||
|
||||
ufCollID UniqueID
|
||||
ufSegID UniqueID
|
||||
ufCollID util.UniqueID
|
||||
ufSegID util.UniqueID
|
||||
ufchanName string
|
||||
ufNor int64
|
||||
|
||||
fCollID UniqueID
|
||||
fSegID UniqueID
|
||||
fCollID util.UniqueID
|
||||
fSegID util.UniqueID
|
||||
fchanName string
|
||||
fNor int64
|
||||
|
||||
@ -176,18 +170,55 @@ func TestDataSyncService_newDataSyncService(t *testing.T) {
|
||||
cm := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir))
|
||||
defer cm.RemoveWithPrefix(ctx, cm.RootPath())
|
||||
|
||||
node := newIDLEDataNodeMock(ctx, schemapb.DataType_Int64)
|
||||
node.allocator = allocator.NewMockAllocator(t)
|
||||
wbManager := writebuffer.NewMockBufferManager(t)
|
||||
wbManager.EXPECT().
|
||||
Register(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil)
|
||||
|
||||
for _, test := range tests {
|
||||
t.Run(test.description, func(t *testing.T) {
|
||||
node.factory = test.inMsgFactory
|
||||
defer node.tryToReleaseFlowgraph(test.chanName)
|
||||
ds, err := newServiceWithEtcdTickler(
|
||||
mockBroker := broker.NewMockBroker(t)
|
||||
mockBroker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).Call.Return(
|
||||
func(_ context.Context, segmentIDs []int64) []*datapb.SegmentInfo {
|
||||
data := map[int64]*datapb.SegmentInfo{
|
||||
test.fSegID: {
|
||||
ID: test.fSegID,
|
||||
CollectionID: test.fCollID,
|
||||
PartitionID: 1,
|
||||
InsertChannel: test.fchanName,
|
||||
State: commonpb.SegmentState_Flushed,
|
||||
},
|
||||
|
||||
test.ufSegID: {
|
||||
ID: test.ufSegID,
|
||||
CollectionID: test.ufCollID,
|
||||
PartitionID: 1,
|
||||
InsertChannel: test.ufchanName,
|
||||
State: commonpb.SegmentState_Flushing,
|
||||
},
|
||||
}
|
||||
return lo.FilterMap(segmentIDs, func(id int64, _ int) (*datapb.SegmentInfo, bool) {
|
||||
item, ok := data[id]
|
||||
return item, ok
|
||||
})
|
||||
}, nil)
|
||||
|
||||
pipelineParams := &util.PipelineParams{
|
||||
Ctx: context.TODO(),
|
||||
Broker: mockBroker,
|
||||
ChunkManager: cm,
|
||||
Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}},
|
||||
SyncMgr: syncmgr.NewMockSyncManager(t),
|
||||
WriteBufferManager: wbManager,
|
||||
Allocator: allocator.NewMockAllocator(t),
|
||||
MsgStreamFactory: test.inMsgFactory,
|
||||
DispClient: msgdispatcher.NewClient(test.inMsgFactory, typeutil.DataNodeRole, 1),
|
||||
}
|
||||
|
||||
ds, err := NewDataSyncService(
|
||||
ctx,
|
||||
node,
|
||||
pipelineParams,
|
||||
getWatchInfo(test),
|
||||
genTestTickler(),
|
||||
util.NewTickler(),
|
||||
)
|
||||
|
||||
if !test.isValidCase {
|
||||
@ -199,122 +230,31 @@ func TestDataSyncService_newDataSyncService(t *testing.T) {
|
||||
|
||||
// start
|
||||
ds.fg = nil
|
||||
ds.start()
|
||||
ds.Start()
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestDataSyncService_newDataSyncService_DuplicatedChannel(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
|
||||
test := &testInfo{
|
||||
true, false, &mockMsgStreamFactory{true, true},
|
||||
1, "by-dev-rootcoord-dml-test_v1",
|
||||
1, 1, "by-dev-rootcoord-dml-test_v1", 0,
|
||||
1, 2, "by-dev-rootcoord-dml-test_v1", 0,
|
||||
"add un-flushed and flushed segments",
|
||||
}
|
||||
cm := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir))
|
||||
defer cm.RemoveWithPrefix(ctx, cm.RootPath())
|
||||
|
||||
watchInfo := getWatchInfo(test)
|
||||
|
||||
node := newIDLEDataNodeMock(ctx, schemapb.DataType_Int64)
|
||||
node.allocator = allocator.NewMockAllocator(t)
|
||||
node.factory = test.inMsgFactory
|
||||
metacache := metacache.NewMockMetaCache(t)
|
||||
metacache.EXPECT().Collection().Return(test.collID)
|
||||
metacache.EXPECT().Schema().Return(watchInfo.GetSchema())
|
||||
node.writeBufferManager.Register(test.chanName, metacache, nil, writebuffer.WithIDAllocator(allocator.NewMockAllocator(t)))
|
||||
ds, err := newServiceWithEtcdTickler(
|
||||
ctx,
|
||||
node,
|
||||
watchInfo,
|
||||
genTestTickler(),
|
||||
)
|
||||
|
||||
assert.Error(t, err)
|
||||
assert.Nil(t, ds)
|
||||
}
|
||||
|
||||
func genBytes() (rawData []byte) {
|
||||
const DIM = 2
|
||||
const N = 1
|
||||
|
||||
// Float vector
|
||||
fvector := [DIM]float32{1, 2}
|
||||
for _, ele := range fvector {
|
||||
buf := make([]byte, 4)
|
||||
common.Endian.PutUint32(buf, math.Float32bits(ele))
|
||||
rawData = append(rawData, buf...)
|
||||
}
|
||||
|
||||
// Binary vector
|
||||
// Dimension of binary vector is 32
|
||||
// size := 4, = 32 / 8
|
||||
bvector := []byte{255, 255, 255, 0}
|
||||
rawData = append(rawData, bvector...)
|
||||
|
||||
// Bool
|
||||
fieldBool := true
|
||||
buf := new(bytes.Buffer)
|
||||
if err := binary.Write(buf, common.Endian, fieldBool); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
rawData = append(rawData, buf.Bytes()...)
|
||||
|
||||
// int8
|
||||
var dataInt8 int8 = 100
|
||||
bint8 := new(bytes.Buffer)
|
||||
if err := binary.Write(bint8, common.Endian, dataInt8); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
rawData = append(rawData, bint8.Bytes()...)
|
||||
log.Debug("Rawdata length:", zap.Int("Length of rawData", len(rawData)))
|
||||
return
|
||||
}
|
||||
|
||||
func TestBytesReader(t *testing.T) {
|
||||
rawData := genBytes()
|
||||
|
||||
// Bytes Reader is able to recording the position
|
||||
rawDataReader := bytes.NewReader(rawData)
|
||||
|
||||
fvector := make([]float32, 2)
|
||||
err := binary.Read(rawDataReader, common.Endian, &fvector)
|
||||
assert.NoError(t, err)
|
||||
assert.ElementsMatch(t, fvector, []float32{1, 2})
|
||||
|
||||
bvector := make([]byte, 4)
|
||||
err = binary.Read(rawDataReader, common.Endian, &bvector)
|
||||
assert.NoError(t, err)
|
||||
assert.ElementsMatch(t, bvector, []byte{255, 255, 255, 0})
|
||||
|
||||
var fieldBool bool
|
||||
err = binary.Read(rawDataReader, common.Endian, &fieldBool)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, true, fieldBool)
|
||||
|
||||
var dataInt8 int8
|
||||
err = binary.Read(rawDataReader, common.Endian, &dataInt8)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, int8(100), dataInt8)
|
||||
}
|
||||
|
||||
func TestGetChannelWithTickler(t *testing.T) {
|
||||
channelName := "by-dev-rootcoord-dml-0"
|
||||
info := getWatchInfoByOpID(100, channelName, datapb.ChannelWatchState_ToWatch)
|
||||
node := newIDLEDataNodeMock(context.Background(), schemapb.DataType_Int64)
|
||||
node.chunkManager = storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir))
|
||||
defer node.chunkManager.RemoveWithPrefix(context.Background(), node.chunkManager.RootPath())
|
||||
info := util.GetWatchInfoByOpID(100, channelName, datapb.ChannelWatchState_ToWatch)
|
||||
chunkManager := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir))
|
||||
defer chunkManager.RemoveWithPrefix(context.Background(), chunkManager.RootPath())
|
||||
|
||||
meta := NewMetaFactory().GetCollectionMeta(1, "test_collection", schemapb.DataType_Int64)
|
||||
broker := broker.NewMockBroker(t)
|
||||
node.broker = broker
|
||||
meta := util.NewMetaFactory().GetCollectionMeta(1, "test_collection", schemapb.DataType_Int64)
|
||||
info.Schema = meta.GetSchema()
|
||||
|
||||
pipelineParams := &util.PipelineParams{
|
||||
Ctx: context.TODO(),
|
||||
Broker: broker.NewMockBroker(t),
|
||||
ChunkManager: chunkManager,
|
||||
Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}},
|
||||
SyncMgr: syncmgr.NewMockSyncManager(t),
|
||||
WriteBufferManager: writebuffer.NewMockBufferManager(t),
|
||||
Allocator: allocator.NewMockAllocator(t),
|
||||
}
|
||||
|
||||
unflushed := []*datapb.SegmentInfo{
|
||||
{
|
||||
ID: 100,
|
||||
@ -349,7 +289,7 @@ func TestGetChannelWithTickler(t *testing.T) {
|
||||
},
|
||||
}
|
||||
|
||||
metaCache, err := getMetaCacheWithTickler(context.TODO(), node, info, newTickler(), unflushed, flushed, nil)
|
||||
metaCache, err := getMetaCacheWithTickler(context.TODO(), pipelineParams, info, util.NewTickler(), unflushed, flushed, nil)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, metaCache)
|
||||
assert.Equal(t, int64(1), metaCache.Collection())
|
||||
@ -359,49 +299,39 @@ func TestGetChannelWithTickler(t *testing.T) {
|
||||
|
||||
type DataSyncServiceSuite struct {
|
||||
suite.Suite
|
||||
MockDataSuiteBase
|
||||
util.MockDataSuiteBase
|
||||
|
||||
node *DataNode // node param
|
||||
chunkManager *mocks.ChunkManager
|
||||
broker *broker.MockBroker
|
||||
allocator *allocator.MockAllocator
|
||||
wbManager *writebuffer.MockBufferManager
|
||||
|
||||
factory *dependency.MockFactory
|
||||
ms *msgstream.MockMsgStream
|
||||
msChan chan *msgstream.MsgPack
|
||||
pipelineParams *util.PipelineParams // node param
|
||||
chunkManager *mocks.ChunkManager
|
||||
broker *broker.MockBroker
|
||||
allocator *allocator.MockAllocator
|
||||
wbManager *writebuffer.MockBufferManager
|
||||
channelCheckpointUpdater *util.ChannelCheckpointUpdater
|
||||
factory *dependency.MockFactory
|
||||
ms *msgstream.MockMsgStream
|
||||
msChan chan *msgstream.MsgPack
|
||||
}
|
||||
|
||||
func (s *DataSyncServiceSuite) SetupSuite() {
|
||||
paramtable.Get().Init(paramtable.NewBaseTable())
|
||||
s.MockDataSuiteBase.prepareData()
|
||||
s.MockDataSuiteBase.PrepareData()
|
||||
}
|
||||
|
||||
func (s *DataSyncServiceSuite) SetupTest() {
|
||||
s.node = &DataNode{}
|
||||
|
||||
s.chunkManager = mocks.NewChunkManager(s.T())
|
||||
|
||||
s.broker = broker.NewMockBroker(s.T())
|
||||
s.broker.EXPECT().UpdateSegmentStatistics(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
|
||||
s.allocator = allocator.NewMockAllocator(s.T())
|
||||
s.wbManager = writebuffer.NewMockBufferManager(s.T())
|
||||
|
||||
s.broker.EXPECT().UpdateSegmentStatistics(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
|
||||
s.node.chunkManager = s.chunkManager
|
||||
s.node.broker = s.broker
|
||||
s.node.allocator = s.allocator
|
||||
s.node.writeBufferManager = s.wbManager
|
||||
s.node.session = &sessionutil.Session{
|
||||
SessionRaw: sessionutil.SessionRaw{
|
||||
ServerID: 1,
|
||||
},
|
||||
}
|
||||
s.node.ctx = context.Background()
|
||||
s.node.channelCheckpointUpdater = newChannelCheckpointUpdater(s.node.broker)
|
||||
paramtable.Get().Save(paramtable.Get().DataNodeCfg.ChannelCheckpointUpdateTickInSeconds.Key, "0.01")
|
||||
defer paramtable.Get().Save(paramtable.Get().DataNodeCfg.ChannelCheckpointUpdateTickInSeconds.Key, "10")
|
||||
go s.node.channelCheckpointUpdater.start()
|
||||
s.msChan = make(chan *msgstream.MsgPack)
|
||||
s.channelCheckpointUpdater = util.NewChannelCheckpointUpdater(s.broker)
|
||||
|
||||
go s.channelCheckpointUpdater.Start()
|
||||
s.msChan = make(chan *msgstream.MsgPack, 1)
|
||||
|
||||
s.factory = dependency.NewMockFactory(s.T())
|
||||
s.ms = msgstream.NewMockMsgStream(s.T())
|
||||
@ -410,18 +340,27 @@ func (s *DataSyncServiceSuite) SetupTest() {
|
||||
s.ms.EXPECT().Chan().Return(s.msChan)
|
||||
s.ms.EXPECT().Close().Return()
|
||||
|
||||
s.node.factory = s.factory
|
||||
s.node.dispClient = msgdispatcher.NewClient(s.factory, typeutil.DataNodeRole, 1)
|
||||
|
||||
s.node.timeTickSender = newTimeTickSender(s.broker, 0)
|
||||
s.pipelineParams = &util.PipelineParams{
|
||||
Ctx: context.TODO(),
|
||||
MsgStreamFactory: s.factory,
|
||||
Broker: s.broker,
|
||||
ChunkManager: s.chunkManager,
|
||||
Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}},
|
||||
CheckpointUpdater: s.channelCheckpointUpdater,
|
||||
SyncMgr: syncmgr.NewMockSyncManager(s.T()),
|
||||
WriteBufferManager: s.wbManager,
|
||||
Allocator: s.allocator,
|
||||
TimeTickSender: util.NewTimeTickSender(s.broker, 0),
|
||||
DispClient: msgdispatcher.NewClient(s.factory, typeutil.DataNodeRole, 1),
|
||||
}
|
||||
}
|
||||
|
||||
func (s *DataSyncServiceSuite) TestStartStop() {
|
||||
var (
|
||||
insertChannelName = fmt.Sprintf("by-dev-rootcoord-dml-%d", rand.Int())
|
||||
|
||||
Factory = &MetaFactory{}
|
||||
collMeta = Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64)
|
||||
Factory = &util.MetaFactory{}
|
||||
collMeta = Factory.GetCollectionMeta(util.UniqueID(0), "coll1", schemapb.DataType_Int64)
|
||||
)
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
@ -486,30 +425,30 @@ func (s *DataSyncServiceSuite) TestStartStop() {
|
||||
},
|
||||
}
|
||||
|
||||
sync, err := newServiceWithEtcdTickler(
|
||||
sync, err := NewDataSyncService(
|
||||
ctx,
|
||||
s.node,
|
||||
s.pipelineParams,
|
||||
watchInfo,
|
||||
genTestTickler(),
|
||||
util.NewTickler(),
|
||||
)
|
||||
s.Require().NoError(err)
|
||||
s.Require().NotNil(sync)
|
||||
|
||||
sync.start()
|
||||
sync.Start()
|
||||
defer sync.close()
|
||||
|
||||
timeRange := TimeRange{
|
||||
timestampMin: 0,
|
||||
timestampMax: math.MaxUint64 - 1,
|
||||
timeRange := util.TimeRange{
|
||||
TimestampMin: 0,
|
||||
TimestampMax: math.MaxUint64 - 1,
|
||||
}
|
||||
|
||||
msgTs := tsoutil.GetCurrentTime()
|
||||
dataFactory := NewDataFactory()
|
||||
dataFactory := util.NewDataFactory()
|
||||
insertMessages := dataFactory.GetMsgStreamTsInsertMsgs(2, insertChannelName, msgTs)
|
||||
|
||||
msgPack := msgstream.MsgPack{
|
||||
BeginTs: timeRange.timestampMin,
|
||||
EndTs: timeRange.timestampMax,
|
||||
BeginTs: timeRange.TimestampMin,
|
||||
EndTs: timeRange.TimestampMax,
|
||||
Msgs: insertMessages,
|
||||
StartPositions: []*msgpb.MsgPosition{{
|
||||
Timestamp: msgTs,
|
||||
@ -533,7 +472,7 @@ func (s *DataSyncServiceSuite) TestStartStop() {
|
||||
TimeTickMsg: msgpb.TimeTickMsg{
|
||||
Base: &commonpb.MsgBase{
|
||||
MsgType: commonpb.MsgType_TimeTick,
|
||||
MsgID: UniqueID(0),
|
||||
MsgID: util.UniqueID(0),
|
||||
Timestamp: tsoutil.GetCurrentTime(),
|
||||
SourceID: 0,
|
||||
},
|
||||
@ -546,7 +485,6 @@ func (s *DataSyncServiceSuite) TestStartStop() {
|
||||
s.wbManager.EXPECT().NotifyCheckpointUpdated(insertChannelName, msgTs).Return().Maybe()
|
||||
|
||||
ch := make(chan struct{})
|
||||
|
||||
s.broker.EXPECT().UpdateChannelCheckpoint(mock.Anything, mock.Anything).RunAndReturn(func(_ context.Context, _ []*msgpb.MsgPosition) error {
|
||||
close(ch)
|
||||
return nil
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"context"
|
||||
@ -29,6 +29,7 @@ import (
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/compaction"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
@ -61,15 +62,15 @@ type ddNode struct {
|
||||
BaseNode
|
||||
|
||||
ctx context.Context
|
||||
collectionID UniqueID
|
||||
collectionID util.UniqueID
|
||||
vChannelName string
|
||||
|
||||
dropMode atomic.Value
|
||||
compactionExecutor compaction.Executor
|
||||
|
||||
// for recovery
|
||||
growingSegInfo map[UniqueID]*datapb.SegmentInfo // segmentID
|
||||
sealedSegInfo map[UniqueID]*datapb.SegmentInfo // segmentID
|
||||
growingSegInfo map[util.UniqueID]*datapb.SegmentInfo // segmentID
|
||||
sealedSegInfo map[util.UniqueID]*datapb.SegmentInfo // segmentID
|
||||
droppedSegmentIDs []int64
|
||||
}
|
||||
|
||||
@ -99,15 +100,15 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
|
||||
}
|
||||
|
||||
if msMsg.IsCloseMsg() {
|
||||
fgMsg := flowGraphMsg{
|
||||
fgMsg := FlowGraphMsg{
|
||||
BaseMsg: flowgraph.NewBaseMsg(true),
|
||||
insertMessages: make([]*msgstream.InsertMsg, 0),
|
||||
timeRange: TimeRange{
|
||||
timestampMin: msMsg.TimestampMin(),
|
||||
timestampMax: msMsg.TimestampMax(),
|
||||
InsertMessages: make([]*msgstream.InsertMsg, 0),
|
||||
TimeRange: util.TimeRange{
|
||||
TimestampMin: msMsg.TimestampMin(),
|
||||
TimestampMax: msMsg.TimestampMax(),
|
||||
},
|
||||
startPositions: msMsg.StartPositions(),
|
||||
endPositions: msMsg.EndPositions(),
|
||||
StartPositions: msMsg.StartPositions(),
|
||||
EndPositions: msMsg.EndPositions(),
|
||||
dropCollection: false,
|
||||
}
|
||||
log.Warn("MsgStream closed", zap.Any("ddNode node", ddn.Name()), zap.String("channel", ddn.vChannelName), zap.Int64("collection", ddn.collectionID))
|
||||
@ -121,7 +122,7 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
|
||||
|
||||
var spans []trace.Span
|
||||
for _, msg := range msMsg.TsMessages() {
|
||||
ctx, sp := startTracer(msg, "DDNode-Operate")
|
||||
ctx, sp := util.StartTracer(msg, "DDNode-Operate")
|
||||
spans = append(spans, sp)
|
||||
msg.SetTraceCtx(ctx)
|
||||
}
|
||||
@ -131,14 +132,14 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
|
||||
}
|
||||
}()
|
||||
|
||||
fgMsg := flowGraphMsg{
|
||||
insertMessages: make([]*msgstream.InsertMsg, 0),
|
||||
timeRange: TimeRange{
|
||||
timestampMin: msMsg.TimestampMin(),
|
||||
timestampMax: msMsg.TimestampMax(),
|
||||
fgMsg := FlowGraphMsg{
|
||||
InsertMessages: make([]*msgstream.InsertMsg, 0),
|
||||
TimeRange: util.TimeRange{
|
||||
TimestampMin: msMsg.TimestampMin(),
|
||||
TimestampMax: msMsg.TimestampMax(),
|
||||
},
|
||||
startPositions: make([]*msgpb.MsgPosition, 0),
|
||||
endPositions: make([]*msgpb.MsgPosition, 0),
|
||||
StartPositions: make([]*msgpb.MsgPosition, 0),
|
||||
EndPositions: make([]*msgpb.MsgPosition, 0),
|
||||
dropCollection: false,
|
||||
}
|
||||
|
||||
@ -180,7 +181,7 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
|
||||
continue
|
||||
}
|
||||
|
||||
rateCol.Add(metricsinfo.InsertConsumeThroughput, float64(proto.Size(&imsg.InsertRequest)))
|
||||
util.RateCol.Add(metricsinfo.InsertConsumeThroughput, float64(proto.Size(&imsg.InsertRequest)))
|
||||
|
||||
metrics.DataNodeConsumeBytesCount.
|
||||
WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.InsertLabel).
|
||||
@ -198,7 +199,7 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
|
||||
zap.Int64("segmentID", imsg.GetSegmentID()),
|
||||
zap.String("channel", ddn.vChannelName),
|
||||
zap.Int("numRows", len(imsg.GetRowIDs())))
|
||||
fgMsg.insertMessages = append(fgMsg.insertMessages, imsg)
|
||||
fgMsg.InsertMessages = append(fgMsg.InsertMessages, imsg)
|
||||
|
||||
case commonpb.MsgType_Delete:
|
||||
dmsg := msg.(*msgstream.DeleteMsg)
|
||||
@ -212,7 +213,7 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
|
||||
}
|
||||
|
||||
log.Debug("DDNode receive delete messages", zap.String("channel", ddn.vChannelName), zap.Int64("numRows", dmsg.NumRows))
|
||||
rateCol.Add(metricsinfo.DeleteConsumeThroughput, float64(proto.Size(&dmsg.DeleteRequest)))
|
||||
util.RateCol.Add(metricsinfo.DeleteConsumeThroughput, float64(proto.Size(&dmsg.DeleteRequest)))
|
||||
|
||||
metrics.DataNodeConsumeBytesCount.
|
||||
WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.DeleteLabel).
|
||||
@ -225,12 +226,12 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
|
||||
metrics.DataNodeConsumeMsgRowsCount.
|
||||
WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.DeleteLabel).
|
||||
Add(float64(dmsg.GetNumRows()))
|
||||
fgMsg.deleteMessages = append(fgMsg.deleteMessages, dmsg)
|
||||
fgMsg.DeleteMessages = append(fgMsg.DeleteMessages, dmsg)
|
||||
}
|
||||
}
|
||||
|
||||
fgMsg.startPositions = append(fgMsg.startPositions, msMsg.StartPositions()...)
|
||||
fgMsg.endPositions = append(fgMsg.endPositions, msMsg.EndPositions()...)
|
||||
fgMsg.StartPositions = append(fgMsg.StartPositions, msMsg.StartPositions()...)
|
||||
fgMsg.EndPositions = append(fgMsg.EndPositions, msMsg.EndPositions()...)
|
||||
|
||||
return []Msg{&fgMsg}
|
||||
}
|
||||
@ -267,7 +268,7 @@ func (ddn *ddNode) tryToFilterSegmentInsertMessages(msg *msgstream.InsertMsg) bo
|
||||
return false
|
||||
}
|
||||
|
||||
func (ddn *ddNode) isDropped(segID UniqueID) bool {
|
||||
func (ddn *ddNode) isDropped(segID util.UniqueID) bool {
|
||||
for _, droppedSegmentID := range ddn.droppedSegmentIDs {
|
||||
if droppedSegmentID == segID {
|
||||
return true
|
||||
@ -280,19 +281,19 @@ func (ddn *ddNode) Close() {
|
||||
log.Info("Flowgraph DD Node closing")
|
||||
}
|
||||
|
||||
func newDDNode(ctx context.Context, collID UniqueID, vChannelName string, droppedSegmentIDs []UniqueID,
|
||||
func newDDNode(ctx context.Context, collID util.UniqueID, vChannelName string, droppedSegmentIDs []util.UniqueID,
|
||||
sealedSegments []*datapb.SegmentInfo, growingSegments []*datapb.SegmentInfo, executor compaction.Executor,
|
||||
) (*ddNode, error) {
|
||||
baseNode := BaseNode{}
|
||||
baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32())
|
||||
baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32())
|
||||
baseNode.SetMaxQueueLength(paramtable.Get().DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32())
|
||||
baseNode.SetMaxParallelism(paramtable.Get().DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32())
|
||||
|
||||
dd := &ddNode{
|
||||
ctx: ctx,
|
||||
BaseNode: baseNode,
|
||||
collectionID: collID,
|
||||
sealedSegInfo: make(map[UniqueID]*datapb.SegmentInfo, len(sealedSegments)),
|
||||
growingSegInfo: make(map[UniqueID]*datapb.SegmentInfo, len(growingSegments)),
|
||||
sealedSegInfo: make(map[util.UniqueID]*datapb.SegmentInfo, len(sealedSegments)),
|
||||
growingSegInfo: make(map[util.UniqueID]*datapb.SegmentInfo, len(growingSegments)),
|
||||
droppedSegmentIDs: droppedSegmentIDs,
|
||||
vChannelName: vChannelName,
|
||||
compactionExecutor: executor,
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"context"
|
||||
@ -27,6 +27,7 @@ import (
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/compaction"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
@ -63,9 +64,9 @@ func TestFlowGraph_DDNode_newDDNode(t *testing.T) {
|
||||
}
|
||||
|
||||
var (
|
||||
collectionID = UniqueID(1)
|
||||
collectionID = util.UniqueID(1)
|
||||
channelName = fmt.Sprintf("by-dev-rootcoord-dml-%s", t.Name())
|
||||
droppedSegIDs = []UniqueID{}
|
||||
droppedSegIDs = []util.UniqueID{}
|
||||
)
|
||||
|
||||
for _, test := range tests {
|
||||
@ -102,11 +103,11 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) {
|
||||
"Invalid input length == 0",
|
||||
},
|
||||
{
|
||||
[]Msg{&flowGraphMsg{}, &flowGraphMsg{}, &flowGraphMsg{}},
|
||||
[]Msg{&FlowGraphMsg{}, &FlowGraphMsg{}, &FlowGraphMsg{}},
|
||||
"Invalid input length == 3",
|
||||
},
|
||||
{
|
||||
[]Msg{&flowGraphMsg{}},
|
||||
[]Msg{&FlowGraphMsg{}},
|
||||
"Invalid input length == 1 but input message is not msgStreamMsg",
|
||||
},
|
||||
}
|
||||
@ -119,9 +120,9 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) {
|
||||
}
|
||||
// valid inputs
|
||||
tests := []struct {
|
||||
ddnCollID UniqueID
|
||||
ddnCollID util.UniqueID
|
||||
|
||||
msgCollID UniqueID
|
||||
msgCollID util.UniqueID
|
||||
expectedChlen int
|
||||
|
||||
description string
|
||||
@ -158,7 +159,7 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) {
|
||||
|
||||
if test.ddnCollID == test.msgCollID {
|
||||
assert.NotEmpty(t, rt)
|
||||
assert.True(t, rt[0].(*flowGraphMsg).dropCollection)
|
||||
assert.True(t, rt[0].(*FlowGraphMsg).dropCollection)
|
||||
} else {
|
||||
assert.NotEmpty(t, rt)
|
||||
}
|
||||
@ -169,22 +170,22 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) {
|
||||
t.Run("Test DDNode Operate DropPartition Msg", func(t *testing.T) {
|
||||
// valid inputs
|
||||
tests := []struct {
|
||||
ddnCollID UniqueID
|
||||
ddnCollID util.UniqueID
|
||||
|
||||
msgCollID UniqueID
|
||||
msgPartID UniqueID
|
||||
expectOutput []UniqueID
|
||||
msgCollID util.UniqueID
|
||||
msgPartID util.UniqueID
|
||||
expectOutput []util.UniqueID
|
||||
|
||||
description string
|
||||
}{
|
||||
{
|
||||
1, 1, 101,
|
||||
[]UniqueID{101},
|
||||
[]util.UniqueID{101},
|
||||
"DropCollectionMsg collID == ddNode collID",
|
||||
},
|
||||
{
|
||||
1, 2, 101,
|
||||
[]UniqueID{},
|
||||
[]util.UniqueID{},
|
||||
"DropCollectionMsg collID != ddNode collID",
|
||||
},
|
||||
}
|
||||
@ -211,7 +212,7 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) {
|
||||
rt := ddn.Operate([]Msg{msgStreamMsg})
|
||||
|
||||
assert.NotEmpty(t, rt)
|
||||
fgMsg, ok := rt[0].(*flowGraphMsg)
|
||||
fgMsg, ok := rt[0].(*FlowGraphMsg)
|
||||
assert.True(t, ok)
|
||||
assert.ElementsMatch(t, test.expectOutput, fgMsg.dropPartitions)
|
||||
})
|
||||
@ -219,27 +220,27 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) {
|
||||
})
|
||||
|
||||
t.Run("Test DDNode Operate and filter insert msg", func(t *testing.T) {
|
||||
var collectionID UniqueID = 1
|
||||
var collectionID util.UniqueID = 1
|
||||
// Prepare ddNode states
|
||||
ddn := ddNode{
|
||||
ctx: context.Background(),
|
||||
collectionID: collectionID,
|
||||
droppedSegmentIDs: []UniqueID{100},
|
||||
droppedSegmentIDs: []util.UniqueID{100},
|
||||
}
|
||||
|
||||
tsMessages := []msgstream.TsMsg{getInsertMsg(100, 10000), getInsertMsg(200, 20000)}
|
||||
var msgStreamMsg Msg = flowgraph.GenerateMsgStreamMsg(tsMessages, 0, 0, nil, nil)
|
||||
|
||||
rt := ddn.Operate([]Msg{msgStreamMsg})
|
||||
assert.Equal(t, 1, len(rt[0].(*flowGraphMsg).insertMessages))
|
||||
assert.Equal(t, 1, len(rt[0].(*FlowGraphMsg).InsertMessages))
|
||||
})
|
||||
|
||||
t.Run("Test DDNode Operate Delete Msg", func(t *testing.T) {
|
||||
tests := []struct {
|
||||
ddnCollID UniqueID
|
||||
inMsgCollID UniqueID
|
||||
ddnCollID util.UniqueID
|
||||
inMsgCollID util.UniqueID
|
||||
|
||||
MsgEndTs Timestamp
|
||||
MsgEndTs util.Timestamp
|
||||
|
||||
expectedRtLen int
|
||||
description string
|
||||
@ -272,7 +273,7 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) {
|
||||
|
||||
// Test
|
||||
rt := ddn.Operate([]Msg{msgStreamMsg})
|
||||
assert.Equal(t, test.expectedRtLen, len(rt[0].(*flowGraphMsg).deleteMessages))
|
||||
assert.Equal(t, test.expectedRtLen, len(rt[0].(*FlowGraphMsg).DeleteMessages))
|
||||
})
|
||||
}
|
||||
})
|
||||
@ -282,16 +283,16 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
tests := []struct {
|
||||
description string
|
||||
|
||||
droppedSegIDs []UniqueID
|
||||
sealedSegInfo map[UniqueID]*datapb.SegmentInfo
|
||||
growingSegInfo map[UniqueID]*datapb.SegmentInfo
|
||||
droppedSegIDs []util.UniqueID
|
||||
sealedSegInfo map[util.UniqueID]*datapb.SegmentInfo
|
||||
growingSegInfo map[util.UniqueID]*datapb.SegmentInfo
|
||||
|
||||
inMsg *msgstream.InsertMsg
|
||||
expected bool
|
||||
}{
|
||||
{
|
||||
"test dropped segments true",
|
||||
[]UniqueID{100},
|
||||
[]util.UniqueID{100},
|
||||
nil,
|
||||
nil,
|
||||
getInsertMsg(100, 10000),
|
||||
@ -299,7 +300,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
},
|
||||
{
|
||||
"test dropped segments true 2",
|
||||
[]UniqueID{100, 101, 102},
|
||||
[]util.UniqueID{100, 101, 102},
|
||||
nil,
|
||||
nil,
|
||||
getInsertMsg(102, 10000),
|
||||
@ -307,8 +308,8 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
},
|
||||
{
|
||||
"test sealed segments msgTs <= segmentTs true",
|
||||
[]UniqueID{},
|
||||
map[UniqueID]*datapb.SegmentInfo{
|
||||
[]util.UniqueID{},
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
200: getSegmentInfo(200, 50000),
|
||||
300: getSegmentInfo(300, 50000),
|
||||
},
|
||||
@ -318,8 +319,8 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
},
|
||||
{
|
||||
"test sealed segments msgTs <= segmentTs true",
|
||||
[]UniqueID{},
|
||||
map[UniqueID]*datapb.SegmentInfo{
|
||||
[]util.UniqueID{},
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
200: getSegmentInfo(200, 50000),
|
||||
300: getSegmentInfo(300, 50000),
|
||||
},
|
||||
@ -329,8 +330,8 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
},
|
||||
{
|
||||
"test sealed segments msgTs > segmentTs false",
|
||||
[]UniqueID{},
|
||||
map[UniqueID]*datapb.SegmentInfo{
|
||||
[]util.UniqueID{},
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
200: getSegmentInfo(200, 50000),
|
||||
300: getSegmentInfo(300, 50000),
|
||||
},
|
||||
@ -340,9 +341,9 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
},
|
||||
{
|
||||
"test growing segments msgTs <= segmentTs true",
|
||||
[]UniqueID{},
|
||||
[]util.UniqueID{},
|
||||
nil,
|
||||
map[UniqueID]*datapb.SegmentInfo{
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
200: getSegmentInfo(200, 50000),
|
||||
300: getSegmentInfo(300, 50000),
|
||||
},
|
||||
@ -351,9 +352,9 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
},
|
||||
{
|
||||
"test growing segments msgTs > segmentTs false",
|
||||
[]UniqueID{},
|
||||
[]util.UniqueID{},
|
||||
nil,
|
||||
map[UniqueID]*datapb.SegmentInfo{
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
200: getSegmentInfo(200, 50000),
|
||||
300: getSegmentInfo(300, 50000),
|
||||
},
|
||||
@ -362,12 +363,12 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
},
|
||||
{
|
||||
"test not exist",
|
||||
[]UniqueID{},
|
||||
map[UniqueID]*datapb.SegmentInfo{
|
||||
[]util.UniqueID{},
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
400: getSegmentInfo(500, 50000),
|
||||
500: getSegmentInfo(400, 50000),
|
||||
},
|
||||
map[UniqueID]*datapb.SegmentInfo{
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
200: getSegmentInfo(200, 50000),
|
||||
300: getSegmentInfo(300, 50000),
|
||||
},
|
||||
@ -377,7 +378,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
// for pChannel reuse on same collection
|
||||
{
|
||||
"test insert msg with different channelName",
|
||||
[]UniqueID{100},
|
||||
[]util.UniqueID{100},
|
||||
nil,
|
||||
nil,
|
||||
getInsertMsgWithChannel(100, 10000, anotherChannelName),
|
||||
@ -405,10 +406,10 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
description string
|
||||
segRemained bool
|
||||
|
||||
segTs Timestamp
|
||||
msgTs Timestamp
|
||||
segTs util.Timestamp
|
||||
msgTs util.Timestamp
|
||||
|
||||
sealedSegInfo map[UniqueID]*datapb.SegmentInfo
|
||||
sealedSegInfo map[util.UniqueID]*datapb.SegmentInfo
|
||||
inMsg *msgstream.InsertMsg
|
||||
msgFiltered bool
|
||||
}{
|
||||
@ -417,7 +418,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
true,
|
||||
50000,
|
||||
10000,
|
||||
map[UniqueID]*datapb.SegmentInfo{
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
100: getSegmentInfo(100, 50000),
|
||||
101: getSegmentInfo(101, 50000),
|
||||
},
|
||||
@ -429,7 +430,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
true,
|
||||
50000,
|
||||
10000,
|
||||
map[UniqueID]*datapb.SegmentInfo{
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
100: getSegmentInfo(100, 50000),
|
||||
101: getSegmentInfo(101, 50000),
|
||||
},
|
||||
@ -441,7 +442,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
false,
|
||||
50000,
|
||||
10000,
|
||||
map[UniqueID]*datapb.SegmentInfo{
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
100: getSegmentInfo(100, 70000),
|
||||
101: getSegmentInfo(101, 50000),
|
||||
},
|
||||
@ -474,14 +475,14 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
description string
|
||||
segRemained bool
|
||||
|
||||
growingSegInfo map[UniqueID]*datapb.SegmentInfo
|
||||
growingSegInfo map[util.UniqueID]*datapb.SegmentInfo
|
||||
inMsg *msgstream.InsertMsg
|
||||
msgFiltered bool
|
||||
}{
|
||||
{
|
||||
"msgTs<segTs",
|
||||
true,
|
||||
map[UniqueID]*datapb.SegmentInfo{
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
100: getSegmentInfo(100, 50000),
|
||||
101: getSegmentInfo(101, 50000),
|
||||
},
|
||||
@ -491,7 +492,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
{
|
||||
"msgTs==segTs",
|
||||
true,
|
||||
map[UniqueID]*datapb.SegmentInfo{
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
100: getSegmentInfo(100, 50000),
|
||||
101: getSegmentInfo(101, 50000),
|
||||
},
|
||||
@ -501,7 +502,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
{
|
||||
"msgTs>segTs",
|
||||
false,
|
||||
map[UniqueID]*datapb.SegmentInfo{
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
100: getSegmentInfo(100, 50000),
|
||||
101: getSegmentInfo(101, 50000),
|
||||
},
|
||||
@ -535,7 +536,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
||||
func TestFlowGraph_DDNode_isDropped(t *testing.T) {
|
||||
tests := []struct {
|
||||
indroppedSegment []*datapb.SegmentInfo
|
||||
inSeg UniqueID
|
||||
inSeg util.UniqueID
|
||||
|
||||
expectedOut bool
|
||||
|
||||
@ -581,18 +582,18 @@ func TestFlowGraph_DDNode_isDropped(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func getSegmentInfo(segmentID UniqueID, ts Timestamp) *datapb.SegmentInfo {
|
||||
func getSegmentInfo(segmentID util.UniqueID, ts util.Timestamp) *datapb.SegmentInfo {
|
||||
return &datapb.SegmentInfo{
|
||||
ID: segmentID,
|
||||
DmlPosition: &msgpb.MsgPosition{Timestamp: ts},
|
||||
}
|
||||
}
|
||||
|
||||
func getInsertMsg(segmentID UniqueID, ts Timestamp) *msgstream.InsertMsg {
|
||||
func getInsertMsg(segmentID util.UniqueID, ts util.Timestamp) *msgstream.InsertMsg {
|
||||
return getInsertMsgWithChannel(segmentID, ts, ddNodeChannelName)
|
||||
}
|
||||
|
||||
func getInsertMsgWithChannel(segmentID UniqueID, ts Timestamp, vChannelName string) *msgstream.InsertMsg {
|
||||
func getInsertMsgWithChannel(segmentID util.UniqueID, ts util.Timestamp, vChannelName string) *msgstream.InsertMsg {
|
||||
return &msgstream.InsertMsg{
|
||||
BaseMsg: msgstream.BaseMsg{EndTimestamp: ts},
|
||||
InsertRequest: msgpb.InsertRequest{
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"context"
|
||||
@ -66,8 +66,8 @@ func newDmInputNode(initCtx context.Context, dispatcherClient msgdispatcher.Clie
|
||||
node := flowgraph.NewInputNode(
|
||||
input,
|
||||
name,
|
||||
Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32(),
|
||||
Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32(),
|
||||
paramtable.Get().DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32(),
|
||||
paramtable.Get().DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32(),
|
||||
typeutil.DataNodeRole,
|
||||
dmNodeConfig.serverID,
|
||||
dmNodeConfig.collectionID,
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"context"
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"context"
|
||||
@ -22,8 +22,7 @@ import (
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
@ -31,14 +30,13 @@ import (
|
||||
)
|
||||
|
||||
type FlowgraphManager interface {
|
||||
AddFlowgraph(ds *dataSyncService)
|
||||
AddandStartWithEtcdTickler(dn *DataNode, vchan *datapb.VchannelInfo, schema *schemapb.CollectionSchema, tickler *etcdTickler) error
|
||||
AddFlowgraph(ds *DataSyncService)
|
||||
RemoveFlowgraph(channel string)
|
||||
ClearFlowgraphs()
|
||||
|
||||
GetFlowgraphService(channel string) (*dataSyncService, bool)
|
||||
GetFlowgraphService(channel string) (*DataSyncService, bool)
|
||||
HasFlowgraph(channel string) bool
|
||||
HasFlowgraphWithOpID(channel string, opID UniqueID) bool
|
||||
HasFlowgraphWithOpID(channel string, opID int64) bool
|
||||
GetFlowgraphCount() int
|
||||
GetCollectionIDs() []int64
|
||||
|
||||
@ -50,58 +48,36 @@ var _ FlowgraphManager = (*fgManagerImpl)(nil)
|
||||
type fgManagerImpl struct {
|
||||
ctx context.Context
|
||||
cancelFunc context.CancelFunc
|
||||
flowgraphs *typeutil.ConcurrentMap[string, *dataSyncService]
|
||||
flowgraphs *typeutil.ConcurrentMap[string, *DataSyncService]
|
||||
}
|
||||
|
||||
func newFlowgraphManager() *fgManagerImpl {
|
||||
func NewFlowgraphManager() *fgManagerImpl {
|
||||
ctx, cancelFunc := context.WithCancel(context.TODO())
|
||||
return &fgManagerImpl{
|
||||
ctx: ctx,
|
||||
cancelFunc: cancelFunc,
|
||||
flowgraphs: typeutil.NewConcurrentMap[string, *dataSyncService](),
|
||||
flowgraphs: typeutil.NewConcurrentMap[string, *DataSyncService](),
|
||||
}
|
||||
}
|
||||
|
||||
func (fm *fgManagerImpl) AddFlowgraph(ds *dataSyncService) {
|
||||
func (fm *fgManagerImpl) AddFlowgraph(ds *DataSyncService) {
|
||||
fm.flowgraphs.Insert(ds.vchannelName, ds)
|
||||
metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Inc()
|
||||
}
|
||||
|
||||
func (fm *fgManagerImpl) AddandStartWithEtcdTickler(dn *DataNode, vchan *datapb.VchannelInfo, schema *schemapb.CollectionSchema, tickler *etcdTickler) error {
|
||||
log := log.With(zap.String("channel", vchan.GetChannelName()))
|
||||
if fm.flowgraphs.Contain(vchan.GetChannelName()) {
|
||||
log.Warn("try to add an existed DataSyncService")
|
||||
return nil
|
||||
}
|
||||
|
||||
dataSyncService, err := newServiceWithEtcdTickler(fm.ctx, dn, &datapb.ChannelWatchInfo{
|
||||
Schema: schema,
|
||||
Vchan: vchan,
|
||||
}, tickler)
|
||||
if err != nil {
|
||||
log.Warn("fail to create new DataSyncService", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
dataSyncService.start()
|
||||
fm.flowgraphs.Insert(vchan.GetChannelName(), dataSyncService)
|
||||
|
||||
metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Inc()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (fm *fgManagerImpl) RemoveFlowgraph(channel string) {
|
||||
if fg, loaded := fm.flowgraphs.Get(channel); loaded {
|
||||
fg.close()
|
||||
fm.flowgraphs.Remove(channel)
|
||||
|
||||
metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Dec()
|
||||
rateCol.removeFlowGraphChannel(channel)
|
||||
util.RateCol.RemoveFlowGraphChannel(channel)
|
||||
}
|
||||
}
|
||||
|
||||
func (fm *fgManagerImpl) ClearFlowgraphs() {
|
||||
log.Info("start drop all flowgraph resources in DataNode")
|
||||
fm.flowgraphs.Range(func(key string, value *dataSyncService) bool {
|
||||
fm.flowgraphs.Range(func(key string, value *DataSyncService) bool {
|
||||
value.GracefullyClose()
|
||||
fm.flowgraphs.GetAndRemove(key)
|
||||
|
||||
@ -110,7 +86,7 @@ func (fm *fgManagerImpl) ClearFlowgraphs() {
|
||||
})
|
||||
}
|
||||
|
||||
func (fm *fgManagerImpl) GetFlowgraphService(channel string) (*dataSyncService, bool) {
|
||||
func (fm *fgManagerImpl) GetFlowgraphService(channel string) (*DataSyncService, bool) {
|
||||
return fm.flowgraphs.Get(channel)
|
||||
}
|
||||
|
||||
@ -119,7 +95,7 @@ func (fm *fgManagerImpl) HasFlowgraph(channel string) bool {
|
||||
return exist
|
||||
}
|
||||
|
||||
func (fm *fgManagerImpl) HasFlowgraphWithOpID(channel string, opID UniqueID) bool {
|
||||
func (fm *fgManagerImpl) HasFlowgraphWithOpID(channel string, opID util.UniqueID) bool {
|
||||
ds, exist := fm.flowgraphs.Get(channel)
|
||||
return exist && ds.opID == opID
|
||||
}
|
||||
@ -131,7 +107,7 @@ func (fm *fgManagerImpl) GetFlowgraphCount() int {
|
||||
|
||||
func (fm *fgManagerImpl) GetCollectionIDs() []int64 {
|
||||
collectionSet := typeutil.UniqueSet{}
|
||||
fm.flowgraphs.Range(func(key string, value *dataSyncService) bool {
|
||||
fm.flowgraphs.Range(func(key string, value *DataSyncService) bool {
|
||||
collectionSet.Insert(value.metacache.Collection())
|
||||
return true
|
||||
})
|
130
internal/datanode/pipeline/flow_graph_manager_test.go
Normal file
130
internal/datanode/pipeline/flow_graph_manager_test.go
Normal file
@ -0,0 +1,130 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/broker"
|
||||
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/datanode/writebuffer"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
)
|
||||
|
||||
func TestMain(t *testing.M) {
|
||||
paramtable.Init()
|
||||
err := util.InitGlobalRateCollector()
|
||||
if err != nil {
|
||||
panic("init test failed, err = " + err.Error())
|
||||
}
|
||||
code := t.Run()
|
||||
os.Exit(code)
|
||||
}
|
||||
|
||||
func TestFlowGraphManager(t *testing.T) {
|
||||
mockBroker := broker.NewMockBroker(t)
|
||||
mockBroker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
mockBroker.EXPECT().SaveBinlogPaths(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
mockBroker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).Return([]*datapb.SegmentInfo{}, nil).Maybe()
|
||||
mockBroker.EXPECT().DropVirtualChannel(mock.Anything, mock.Anything).Return(nil, nil).Maybe()
|
||||
mockBroker.EXPECT().UpdateChannelCheckpoint(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
|
||||
wbm := writebuffer.NewMockBufferManager(t)
|
||||
wbm.EXPECT().Register(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil)
|
||||
|
||||
dispClient := msgdispatcher.NewMockClient(t)
|
||||
dispClient.EXPECT().Register(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(make(chan *msgstream.MsgPack), nil)
|
||||
dispClient.EXPECT().Deregister(mock.Anything)
|
||||
|
||||
pipelineParams := &util.PipelineParams{
|
||||
Ctx: context.TODO(),
|
||||
Broker: mockBroker,
|
||||
Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 0}},
|
||||
CheckpointUpdater: util.NewChannelCheckpointUpdater(mockBroker),
|
||||
SyncMgr: syncmgr.NewMockSyncManager(t),
|
||||
WriteBufferManager: wbm,
|
||||
Allocator: allocator.NewMockAllocator(t),
|
||||
DispClient: dispClient,
|
||||
}
|
||||
|
||||
fm := NewFlowgraphManager()
|
||||
|
||||
chanWatchInfo := generateChannelWatchInfo()
|
||||
ds, err := NewDataSyncService(
|
||||
context.TODO(),
|
||||
pipelineParams,
|
||||
chanWatchInfo,
|
||||
util.NewTickler(),
|
||||
)
|
||||
assert.NoError(t, err)
|
||||
|
||||
fm.AddFlowgraph(ds)
|
||||
assert.True(t, fm.HasFlowgraph(chanWatchInfo.Vchan.ChannelName))
|
||||
ds, ret := fm.GetFlowgraphService(chanWatchInfo.Vchan.ChannelName)
|
||||
assert.True(t, ret)
|
||||
assert.Equal(t, chanWatchInfo.Vchan.ChannelName, ds.vchannelName)
|
||||
|
||||
fm.RemoveFlowgraph(chanWatchInfo.Vchan.ChannelName)
|
||||
assert.False(t, fm.HasFlowgraph(chanWatchInfo.Vchan.ChannelName))
|
||||
|
||||
fm.ClearFlowgraphs()
|
||||
assert.Equal(t, fm.GetFlowgraphCount(), 0)
|
||||
}
|
||||
|
||||
func generateChannelWatchInfo() *datapb.ChannelWatchInfo {
|
||||
collectionID := int64(rand.Uint32())
|
||||
dmChannelName := fmt.Sprintf("%s_%d", "fake-ch-", collectionID)
|
||||
schema := &schemapb.CollectionSchema{
|
||||
Name: fmt.Sprintf("%s_%d", "collection_", collectionID),
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64},
|
||||
{FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64},
|
||||
{FieldID: common.StartOfUserFieldID, DataType: schemapb.DataType_Int64, IsPrimaryKey: true, Name: "pk"},
|
||||
{FieldID: common.StartOfUserFieldID + 1, DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
}},
|
||||
},
|
||||
}
|
||||
vchan := &datapb.VchannelInfo{
|
||||
CollectionID: collectionID,
|
||||
ChannelName: dmChannelName,
|
||||
UnflushedSegmentIds: []int64{},
|
||||
FlushedSegmentIds: []int64{},
|
||||
}
|
||||
|
||||
return &datapb.ChannelWatchInfo{
|
||||
Vchan: vchan,
|
||||
State: datapb.ChannelWatchState_WatchSuccess,
|
||||
Schema: schema,
|
||||
}
|
||||
}
|
@ -14,10 +14,11 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
@ -42,38 +43,24 @@ type (
|
||||
Blob = storage.Blob
|
||||
)
|
||||
|
||||
type flowGraphMsg struct {
|
||||
type FlowGraphMsg struct {
|
||||
BaseMsg
|
||||
insertMessages []*msgstream.InsertMsg
|
||||
deleteMessages []*msgstream.DeleteMsg
|
||||
timeRange TimeRange
|
||||
startPositions []*msgpb.MsgPosition
|
||||
endPositions []*msgpb.MsgPosition
|
||||
InsertMessages []*msgstream.InsertMsg
|
||||
DeleteMessages []*msgstream.DeleteMsg
|
||||
TimeRange util.TimeRange
|
||||
StartPositions []*msgpb.MsgPosition
|
||||
EndPositions []*msgpb.MsgPosition
|
||||
|
||||
// segmentsToSync is the signal used by insertBufferNode to notify deleteNode to flush
|
||||
segmentsToSync []UniqueID
|
||||
segmentsToSync []util.UniqueID
|
||||
dropCollection bool
|
||||
dropPartitions []UniqueID
|
||||
dropPartitions []util.UniqueID
|
||||
}
|
||||
|
||||
func (fgMsg *flowGraphMsg) TimeTick() Timestamp {
|
||||
return fgMsg.timeRange.timestampMax
|
||||
func (fgMsg *FlowGraphMsg) TimeTick() util.Timestamp {
|
||||
return fgMsg.TimeRange.TimestampMax
|
||||
}
|
||||
|
||||
func (fgMsg *flowGraphMsg) IsClose() bool {
|
||||
func (fgMsg *FlowGraphMsg) IsClose() bool {
|
||||
return fgMsg.BaseMsg.IsCloseMsg()
|
||||
}
|
||||
|
||||
// flush Msg is used in flowgraph insertBufferNode to flush the given segment
|
||||
type flushMsg struct {
|
||||
msgID UniqueID
|
||||
timestamp Timestamp
|
||||
segmentID UniqueID
|
||||
collectionID UniqueID
|
||||
// isFlush illustrates if this is a flush or normal sync
|
||||
isFlush bool
|
||||
}
|
||||
|
||||
type resendTTMsg struct {
|
||||
msgID UniqueID
|
||||
segmentIDs []UniqueID
|
||||
}
|
@ -14,17 +14,19 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
)
|
||||
|
||||
func TestInsertMsg_TimeTick(te *testing.T) {
|
||||
tests := []struct {
|
||||
timeTimestanpMax Timestamp
|
||||
timeTimestanpMax util.Timestamp
|
||||
|
||||
description string
|
||||
}{
|
||||
@ -34,7 +36,7 @@ func TestInsertMsg_TimeTick(te *testing.T) {
|
||||
|
||||
for _, test := range tests {
|
||||
te.Run(test.description, func(t *testing.T) {
|
||||
fgMsg := &flowGraphMsg{timeRange: TimeRange{timestampMax: test.timeTimestanpMax}}
|
||||
fgMsg := &FlowGraphMsg{TimeRange: util.TimeRange{TimestampMax: test.timeTimestanpMax}}
|
||||
assert.Equal(t, test.timeTimestanpMax, fgMsg.TimeTick())
|
||||
})
|
||||
}
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"sync/atomic"
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
@ -26,6 +26,7 @@ import (
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/metacache"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/datanode/writebuffer"
|
||||
"github.com/milvus-io/milvus/internal/util/flowgraph"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
@ -42,7 +43,7 @@ type ttNode struct {
|
||||
metacache metacache.MetaCache
|
||||
writeBufferManager writebuffer.BufferManager
|
||||
lastUpdateTime *atomic.Time
|
||||
cpUpdater *channelCheckpointUpdater
|
||||
cpUpdater *util.ChannelCheckpointUpdater
|
||||
dropMode *atomic.Bool
|
||||
}
|
||||
|
||||
@ -55,7 +56,7 @@ func (ttn *ttNode) IsValidInMsg(in []Msg) bool {
|
||||
if !ttn.BaseNode.IsValidInMsg(in) {
|
||||
return false
|
||||
}
|
||||
_, ok := in[0].(*flowGraphMsg)
|
||||
_, ok := in[0].(*FlowGraphMsg)
|
||||
if !ok {
|
||||
log.Warn("type assertion failed for flowGraphMsg", zap.String("name", reflect.TypeOf(in[0]).Name()))
|
||||
return false
|
||||
@ -68,7 +69,7 @@ func (ttn *ttNode) Close() {
|
||||
|
||||
// Operate handles input messages, implementing flowgraph.Node
|
||||
func (ttn *ttNode) Operate(in []Msg) []Msg {
|
||||
fgMsg := in[0].(*flowGraphMsg)
|
||||
fgMsg := in[0].(*FlowGraphMsg)
|
||||
if fgMsg.dropCollection {
|
||||
ttn.dropMode.Store(true)
|
||||
}
|
||||
@ -80,9 +81,9 @@ func (ttn *ttNode) Operate(in []Msg) []Msg {
|
||||
return []Msg{}
|
||||
}
|
||||
|
||||
curTs, _ := tsoutil.ParseTS(fgMsg.timeRange.timestampMax)
|
||||
curTs, _ := tsoutil.ParseTS(fgMsg.TimeRange.TimestampMax)
|
||||
if fgMsg.IsCloseMsg() {
|
||||
if len(fgMsg.endPositions) > 0 {
|
||||
if len(fgMsg.EndPositions) > 0 {
|
||||
channelPos, _, err := ttn.writeBufferManager.GetCheckpoint(ttn.vChannelName)
|
||||
if err != nil {
|
||||
log.Warn("channel removed", zap.String("channel", ttn.vChannelName), zap.Error(err))
|
||||
@ -128,10 +129,10 @@ func (ttn *ttNode) updateChannelCP(channelPos *msgpb.MsgPosition, curTs time.Tim
|
||||
ttn.lastUpdateTime.Store(curTs)
|
||||
}
|
||||
|
||||
func newTTNode(config *nodeConfig, wbManager writebuffer.BufferManager, cpUpdater *channelCheckpointUpdater) (*ttNode, error) {
|
||||
func newTTNode(config *nodeConfig, wbManager writebuffer.BufferManager, cpUpdater *util.ChannelCheckpointUpdater) (*ttNode, error) {
|
||||
baseNode := BaseNode{}
|
||||
baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32())
|
||||
baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32())
|
||||
baseNode.SetMaxQueueLength(paramtable.Get().DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32())
|
||||
baseNode.SetMaxParallelism(paramtable.Get().DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32())
|
||||
|
||||
tt := &ttNode{
|
||||
BaseNode: baseNode,
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"sync"
|
||||
@ -24,10 +24,12 @@ import (
|
||||
"go.uber.org/zap"
|
||||
"golang.org/x/exp/maps"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
)
|
||||
|
||||
type sendTimeTick func(Timestamp, []int64) error
|
||||
type sendTimeTick func(util.Timestamp, []int64) error
|
||||
|
||||
// mergedTimeTickerSender reduces time ticker sending rate when datanode is doing `fast-forwarding`
|
||||
// it makes sure time ticker send at most 10 times a second (1tick/100millisecond)
|
||||
@ -46,11 +48,6 @@ type mergedTimeTickerSender struct {
|
||||
closeOnce sync.Once
|
||||
}
|
||||
|
||||
var (
|
||||
uniqueMergedTimeTickerSender *mergedTimeTickerSender
|
||||
getUniqueMergedTimeTickerSender sync.Once
|
||||
)
|
||||
|
||||
func newUniqueMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender {
|
||||
return &mergedTimeTickerSender{
|
||||
ts: 0, // 0 for not tt send
|
||||
@ -61,17 +58,7 @@ func newUniqueMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender
|
||||
}
|
||||
}
|
||||
|
||||
func getOrCreateMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender {
|
||||
getUniqueMergedTimeTickerSender.Do(func() {
|
||||
uniqueMergedTimeTickerSender = newUniqueMergedTimeTickerSender(send)
|
||||
uniqueMergedTimeTickerSender.wg.Add(2)
|
||||
go uniqueMergedTimeTickerSender.tick()
|
||||
go uniqueMergedTimeTickerSender.work()
|
||||
})
|
||||
return uniqueMergedTimeTickerSender
|
||||
}
|
||||
|
||||
func (mt *mergedTimeTickerSender) bufferTs(ts Timestamp, segmentIDs []int64) {
|
||||
func (mt *mergedTimeTickerSender) bufferTs(ts util.Timestamp, segmentIDs []int64) {
|
||||
mt.mu.Lock()
|
||||
defer mt.mu.Unlock()
|
||||
mt.ts = ts
|
||||
@ -83,7 +70,7 @@ func (mt *mergedTimeTickerSender) bufferTs(ts Timestamp, segmentIDs []int64) {
|
||||
func (mt *mergedTimeTickerSender) tick() {
|
||||
defer mt.wg.Done()
|
||||
// this duration might be configuable in the future
|
||||
t := time.NewTicker(Params.DataNodeCfg.DataNodeTimeTickInterval.GetAsDuration(time.Millisecond)) // 500 millisecond
|
||||
t := time.NewTicker(paramtable.Get().DataNodeCfg.DataNodeTimeTickInterval.GetAsDuration(time.Millisecond)) // 500 millisecond
|
||||
defer t.Stop()
|
||||
for {
|
||||
select {
|
@ -1,4 +1,4 @@
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"context"
|
||||
@ -11,6 +11,7 @@ import (
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/metacache"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/datanode/writebuffer"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
@ -27,7 +28,7 @@ type writeNode struct {
|
||||
}
|
||||
|
||||
func (wNode *writeNode) Operate(in []Msg) []Msg {
|
||||
fgMsg := in[0].(*flowGraphMsg)
|
||||
fgMsg := in[0].(*FlowGraphMsg)
|
||||
|
||||
// close msg, ignore all data
|
||||
if fgMsg.IsCloseMsg() {
|
||||
@ -35,31 +36,31 @@ func (wNode *writeNode) Operate(in []Msg) []Msg {
|
||||
}
|
||||
|
||||
// replace pchannel with vchannel
|
||||
startPositions := make([]*msgpb.MsgPosition, 0, len(fgMsg.startPositions))
|
||||
for idx := range fgMsg.startPositions {
|
||||
pos := proto.Clone(fgMsg.startPositions[idx]).(*msgpb.MsgPosition)
|
||||
startPositions := make([]*msgpb.MsgPosition, 0, len(fgMsg.StartPositions))
|
||||
for idx := range fgMsg.StartPositions {
|
||||
pos := proto.Clone(fgMsg.StartPositions[idx]).(*msgpb.MsgPosition)
|
||||
pos.ChannelName = wNode.channelName
|
||||
startPositions = append(startPositions, pos)
|
||||
}
|
||||
fgMsg.startPositions = startPositions
|
||||
endPositions := make([]*msgpb.MsgPosition, 0, len(fgMsg.endPositions))
|
||||
for idx := range fgMsg.endPositions {
|
||||
pos := proto.Clone(fgMsg.endPositions[idx]).(*msgpb.MsgPosition)
|
||||
fgMsg.StartPositions = startPositions
|
||||
endPositions := make([]*msgpb.MsgPosition, 0, len(fgMsg.EndPositions))
|
||||
for idx := range fgMsg.EndPositions {
|
||||
pos := proto.Clone(fgMsg.EndPositions[idx]).(*msgpb.MsgPosition)
|
||||
pos.ChannelName = wNode.channelName
|
||||
endPositions = append(endPositions, pos)
|
||||
}
|
||||
fgMsg.endPositions = endPositions
|
||||
fgMsg.EndPositions = endPositions
|
||||
|
||||
if len(fgMsg.startPositions) == 0 {
|
||||
if len(fgMsg.StartPositions) == 0 {
|
||||
return []Msg{}
|
||||
}
|
||||
if len(fgMsg.endPositions) == 0 {
|
||||
if len(fgMsg.EndPositions) == 0 {
|
||||
return []Msg{}
|
||||
}
|
||||
|
||||
var spans []trace.Span
|
||||
for _, msg := range fgMsg.insertMessages {
|
||||
ctx, sp := startTracer(msg, "WriteNode")
|
||||
for _, msg := range fgMsg.InsertMessages {
|
||||
ctx, sp := util.StartTracer(msg, "WriteNode")
|
||||
spans = append(spans, sp)
|
||||
msg.SetTraceCtx(ctx)
|
||||
}
|
||||
@ -69,16 +70,16 @@ func (wNode *writeNode) Operate(in []Msg) []Msg {
|
||||
}
|
||||
}()
|
||||
|
||||
start, end := fgMsg.startPositions[0], fgMsg.endPositions[0]
|
||||
start, end := fgMsg.StartPositions[0], fgMsg.EndPositions[0]
|
||||
|
||||
err := wNode.wbManager.BufferData(wNode.channelName, fgMsg.insertMessages, fgMsg.deleteMessages, start, end)
|
||||
err := wNode.wbManager.BufferData(wNode.channelName, fgMsg.InsertMessages, fgMsg.DeleteMessages, start, end)
|
||||
if err != nil {
|
||||
log.Error("failed to buffer data", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
|
||||
stats := lo.FilterMap(
|
||||
lo.Keys(lo.SliceToMap(fgMsg.insertMessages, func(msg *msgstream.InsertMsg) (int64, struct{}) { return msg.GetSegmentID(), struct{}{} })),
|
||||
lo.Keys(lo.SliceToMap(fgMsg.InsertMessages, func(msg *msgstream.InsertMsg) (int64, struct{}) { return msg.GetSegmentID(), struct{}{} })),
|
||||
func(id int64, _ int) (*commonpb.SegmentStats, bool) {
|
||||
segInfo, ok := wNode.metacache.GetSegmentByID(id)
|
||||
if !ok {
|
||||
@ -91,12 +92,12 @@ func (wNode *writeNode) Operate(in []Msg) []Msg {
|
||||
}, true
|
||||
})
|
||||
|
||||
wNode.updater.update(wNode.channelName, end.GetTimestamp(), stats)
|
||||
wNode.updater.Update(wNode.channelName, end.GetTimestamp(), stats)
|
||||
|
||||
res := flowGraphMsg{
|
||||
timeRange: fgMsg.timeRange,
|
||||
startPositions: fgMsg.startPositions,
|
||||
endPositions: fgMsg.endPositions,
|
||||
res := FlowGraphMsg{
|
||||
TimeRange: fgMsg.TimeRange,
|
||||
StartPositions: fgMsg.StartPositions,
|
||||
EndPositions: fgMsg.EndPositions,
|
||||
dropCollection: fgMsg.dropCollection,
|
||||
}
|
||||
|
@ -1,13 +1,8 @@
|
||||
// Code generated by mockery v2.32.4. DO NOT EDIT.
|
||||
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
datapb "github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
mock "github.com/stretchr/testify/mock"
|
||||
|
||||
schemapb "github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
)
|
||||
import mock "github.com/stretchr/testify/mock"
|
||||
|
||||
// MockFlowgraphManager is an autogenerated mock type for the FlowgraphManager type
|
||||
type MockFlowgraphManager struct {
|
||||
@ -23,7 +18,7 @@ func (_m *MockFlowgraphManager) EXPECT() *MockFlowgraphManager_Expecter {
|
||||
}
|
||||
|
||||
// AddFlowgraph provides a mock function with given fields: ds
|
||||
func (_m *MockFlowgraphManager) AddFlowgraph(ds *dataSyncService) {
|
||||
func (_m *MockFlowgraphManager) AddFlowgraph(ds *DataSyncService) {
|
||||
_m.Called(ds)
|
||||
}
|
||||
|
||||
@ -33,14 +28,14 @@ type MockFlowgraphManager_AddFlowgraph_Call struct {
|
||||
}
|
||||
|
||||
// AddFlowgraph is a helper method to define mock.On call
|
||||
// - ds *dataSyncService
|
||||
// - ds *DataSyncService
|
||||
func (_e *MockFlowgraphManager_Expecter) AddFlowgraph(ds interface{}) *MockFlowgraphManager_AddFlowgraph_Call {
|
||||
return &MockFlowgraphManager_AddFlowgraph_Call{Call: _e.mock.On("AddFlowgraph", ds)}
|
||||
}
|
||||
|
||||
func (_c *MockFlowgraphManager_AddFlowgraph_Call) Run(run func(ds *dataSyncService)) *MockFlowgraphManager_AddFlowgraph_Call {
|
||||
func (_c *MockFlowgraphManager_AddFlowgraph_Call) Run(run func(ds *DataSyncService)) *MockFlowgraphManager_AddFlowgraph_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(*dataSyncService))
|
||||
run(args[0].(*DataSyncService))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
@ -50,52 +45,7 @@ func (_c *MockFlowgraphManager_AddFlowgraph_Call) Return() *MockFlowgraphManager
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockFlowgraphManager_AddFlowgraph_Call) RunAndReturn(run func(*dataSyncService)) *MockFlowgraphManager_AddFlowgraph_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
|
||||
// AddandStartWithEtcdTickler provides a mock function with given fields: dn, vchan, schema, tickler
|
||||
func (_m *MockFlowgraphManager) AddandStartWithEtcdTickler(dn *DataNode, vchan *datapb.VchannelInfo, schema *schemapb.CollectionSchema, tickler *etcdTickler) error {
|
||||
ret := _m.Called(dn, vchan, schema, tickler)
|
||||
|
||||
var r0 error
|
||||
if rf, ok := ret.Get(0).(func(*DataNode, *datapb.VchannelInfo, *schemapb.CollectionSchema, *etcdTickler) error); ok {
|
||||
r0 = rf(dn, vchan, schema, tickler)
|
||||
} else {
|
||||
r0 = ret.Error(0)
|
||||
}
|
||||
|
||||
return r0
|
||||
}
|
||||
|
||||
// MockFlowgraphManager_AddandStartWithEtcdTickler_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'AddandStartWithEtcdTickler'
|
||||
type MockFlowgraphManager_AddandStartWithEtcdTickler_Call struct {
|
||||
*mock.Call
|
||||
}
|
||||
|
||||
// AddandStartWithEtcdTickler is a helper method to define mock.On call
|
||||
// - dn *DataNode
|
||||
// - vchan *datapb.VchannelInfo
|
||||
// - schema *schemapb.CollectionSchema
|
||||
// - tickler *etcdTickler
|
||||
func (_e *MockFlowgraphManager_Expecter) AddandStartWithEtcdTickler(dn interface{}, vchan interface{}, schema interface{}, tickler interface{}) *MockFlowgraphManager_AddandStartWithEtcdTickler_Call {
|
||||
return &MockFlowgraphManager_AddandStartWithEtcdTickler_Call{Call: _e.mock.On("AddandStartWithEtcdTickler", dn, vchan, schema, tickler)}
|
||||
}
|
||||
|
||||
func (_c *MockFlowgraphManager_AddandStartWithEtcdTickler_Call) Run(run func(dn *DataNode, vchan *datapb.VchannelInfo, schema *schemapb.CollectionSchema, tickler *etcdTickler)) *MockFlowgraphManager_AddandStartWithEtcdTickler_Call {
|
||||
_c.Call.Run(func(args mock.Arguments) {
|
||||
run(args[0].(*DataNode), args[1].(*datapb.VchannelInfo), args[2].(*schemapb.CollectionSchema), args[3].(*etcdTickler))
|
||||
})
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockFlowgraphManager_AddandStartWithEtcdTickler_Call) Return(_a0 error) *MockFlowgraphManager_AddandStartWithEtcdTickler_Call {
|
||||
_c.Call.Return(_a0)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockFlowgraphManager_AddandStartWithEtcdTickler_Call) RunAndReturn(run func(*DataNode, *datapb.VchannelInfo, *schemapb.CollectionSchema, *etcdTickler) error) *MockFlowgraphManager_AddandStartWithEtcdTickler_Call {
|
||||
func (_c *MockFlowgraphManager_AddFlowgraph_Call) RunAndReturn(run func(*DataSyncService)) *MockFlowgraphManager_AddFlowgraph_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
||||
@ -249,19 +199,19 @@ func (_c *MockFlowgraphManager_GetFlowgraphCount_Call) RunAndReturn(run func() i
|
||||
}
|
||||
|
||||
// GetFlowgraphService provides a mock function with given fields: channel
|
||||
func (_m *MockFlowgraphManager) GetFlowgraphService(channel string) (*dataSyncService, bool) {
|
||||
func (_m *MockFlowgraphManager) GetFlowgraphService(channel string) (*DataSyncService, bool) {
|
||||
ret := _m.Called(channel)
|
||||
|
||||
var r0 *dataSyncService
|
||||
var r0 *DataSyncService
|
||||
var r1 bool
|
||||
if rf, ok := ret.Get(0).(func(string) (*dataSyncService, bool)); ok {
|
||||
if rf, ok := ret.Get(0).(func(string) (*DataSyncService, bool)); ok {
|
||||
return rf(channel)
|
||||
}
|
||||
if rf, ok := ret.Get(0).(func(string) *dataSyncService); ok {
|
||||
if rf, ok := ret.Get(0).(func(string) *DataSyncService); ok {
|
||||
r0 = rf(channel)
|
||||
} else {
|
||||
if ret.Get(0) != nil {
|
||||
r0 = ret.Get(0).(*dataSyncService)
|
||||
r0 = ret.Get(0).(*DataSyncService)
|
||||
}
|
||||
}
|
||||
|
||||
@ -292,12 +242,12 @@ func (_c *MockFlowgraphManager_GetFlowgraphService_Call) Run(run func(channel st
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockFlowgraphManager_GetFlowgraphService_Call) Return(_a0 *dataSyncService, _a1 bool) *MockFlowgraphManager_GetFlowgraphService_Call {
|
||||
func (_c *MockFlowgraphManager_GetFlowgraphService_Call) Return(_a0 *DataSyncService, _a1 bool) *MockFlowgraphManager_GetFlowgraphService_Call {
|
||||
_c.Call.Return(_a0, _a1)
|
||||
return _c
|
||||
}
|
||||
|
||||
func (_c *MockFlowgraphManager_GetFlowgraphService_Call) RunAndReturn(run func(string) (*dataSyncService, bool)) *MockFlowgraphManager_GetFlowgraphService_Call {
|
||||
func (_c *MockFlowgraphManager_GetFlowgraphService_Call) RunAndReturn(run func(string) (*DataSyncService, bool)) *MockFlowgraphManager_GetFlowgraphService_Call {
|
||||
_c.Call.Return(run)
|
||||
return _c
|
||||
}
|
@ -1,4 +1,4 @@
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
@ -8,6 +8,7 @@ import (
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/pkg/metrics"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
|
||||
@ -16,7 +17,7 @@ import (
|
||||
)
|
||||
|
||||
type statsUpdater interface {
|
||||
update(channel string, ts Timestamp, stats []*commonpb.SegmentStats)
|
||||
Update(channel string, ts util.Timestamp, stats []*commonpb.SegmentStats)
|
||||
}
|
||||
|
||||
// mqStatsUpdater is the wrapper of mergedTimeTickSender
|
||||
@ -40,7 +41,7 @@ func newMqStatsUpdater(config *nodeConfig, producer msgstream.MsgStream) statsUp
|
||||
return updater
|
||||
}
|
||||
|
||||
func (u *mqStatsUpdater) send(ts Timestamp, segmentIDs []int64) error {
|
||||
func (u *mqStatsUpdater) send(ts util.Timestamp, segmentIDs []int64) error {
|
||||
u.mut.Lock()
|
||||
defer u.mut.Unlock()
|
||||
stats := lo.Map(segmentIDs, func(id int64, _ int) *commonpb.SegmentStats {
|
||||
@ -86,7 +87,7 @@ func (u *mqStatsUpdater) send(ts Timestamp, segmentIDs []int64) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (u *mqStatsUpdater) update(channel string, ts Timestamp, stats []*commonpb.SegmentStats) {
|
||||
func (u *mqStatsUpdater) Update(channel string, ts util.Timestamp, stats []*commonpb.SegmentStats) {
|
||||
u.mut.Lock()
|
||||
defer u.mut.Unlock()
|
||||
segmentIDs := lo.Map(stats, func(stats *commonpb.SegmentStats, _ int) int64 { return stats.SegmentID })
|
@ -1,4 +1,4 @@
|
||||
package datanode
|
||||
package pipeline
|
||||
|
||||
import (
|
||||
"testing"
|
@ -32,7 +32,6 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/datanode/importv2"
|
||||
"github.com/milvus-io/milvus/internal/datanode/io"
|
||||
"github.com/milvus-io/milvus/internal/datanode/metacache"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
@ -306,7 +305,7 @@ func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegments
|
||||
allSegments[segID] = struct{}{}
|
||||
}
|
||||
|
||||
missingSegments := ds.metacache.DetectMissingSegments(allSegments)
|
||||
missingSegments := ds.GetMetaCache().DetectMissingSegments(allSegments)
|
||||
|
||||
newSegments := make([]*datapb.SyncSegmentInfo, 0, len(missingSegments))
|
||||
futures := make([]*conc.Future[any], 0, len(missingSegments))
|
||||
@ -323,7 +322,7 @@ func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegments
|
||||
log.Warn("failed to DecompressBinLog", zap.Error(err))
|
||||
return val, err
|
||||
}
|
||||
pks, err := util.LoadStats(ctx, node.chunkManager, ds.metacache.Schema(), newSeg.GetSegmentId(), []*datapb.FieldBinlog{newSeg.GetPkStatsLog()})
|
||||
pks, err := compaction.LoadStats(ctx, node.chunkManager, ds.GetMetaCache().Schema(), newSeg.GetSegmentId(), []*datapb.FieldBinlog{newSeg.GetPkStatsLog()})
|
||||
if err != nil {
|
||||
log.Warn("failed to load segment stats log", zap.Error(err))
|
||||
return val, err
|
||||
@ -343,7 +342,7 @@ func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegments
|
||||
return future.Value().(*metacache.BloomFilterSet)
|
||||
})
|
||||
|
||||
ds.metacache.UpdateSegmentView(req.GetPartitionId(), newSegments, newSegmentsBF, allSegments)
|
||||
ds.GetMetaCache().UpdateSegmentView(req.GetPartitionId(), newSegments, newSegmentsBF, allSegments)
|
||||
return merr.Success(), nil
|
||||
}
|
||||
|
||||
|
@ -36,6 +36,8 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/datanode/broker"
|
||||
"github.com/milvus-io/milvus/internal/datanode/compaction"
|
||||
"github.com/milvus-io/milvus/internal/datanode/metacache"
|
||||
"github.com/milvus-io/milvus/internal/datanode/pipeline"
|
||||
"github.com/milvus-io/milvus/internal/datanode/util"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
@ -77,7 +79,7 @@ func (s *DataNodeServicesSuite) SetupSuite() {
|
||||
}
|
||||
|
||||
func (s *DataNodeServicesSuite) SetupTest() {
|
||||
s.node = newIDLEDataNodeMock(s.ctx, schemapb.DataType_Int64)
|
||||
s.node = NewIDLEDataNodeMock(s.ctx, schemapb.DataType_Int64)
|
||||
s.node.SetEtcdClient(s.etcdCli)
|
||||
|
||||
err := s.node.Init()
|
||||
@ -292,18 +294,25 @@ func (s *DataNodeServicesSuite) TestFlushSegments() {
|
||||
FlushedSegmentIds: []int64{},
|
||||
}
|
||||
|
||||
err := s.node.flowgraphManager.AddandStartWithEtcdTickler(s.node, vchan, schema, genTestTickler())
|
||||
s.Require().NoError(err)
|
||||
|
||||
fgservice, ok := s.node.flowgraphManager.GetFlowgraphService(dmChannelName)
|
||||
s.Require().True(ok)
|
||||
chanWathInfo := &datapb.ChannelWatchInfo{
|
||||
Vchan: vchan,
|
||||
State: datapb.ChannelWatchState_WatchSuccess,
|
||||
Schema: schema,
|
||||
}
|
||||
|
||||
metaCache := metacache.NewMockMetaCache(s.T())
|
||||
metaCache.EXPECT().Collection().Return(1).Maybe()
|
||||
metaCache.EXPECT().Schema().Return(schema).Maybe()
|
||||
s.node.writeBufferManager.Register(dmChannelName, metaCache, nil)
|
||||
|
||||
fgservice.metacache.AddSegment(&datapb.SegmentInfo{
|
||||
ds, err := pipeline.NewDataSyncService(context.TODO(), getPipelineParams(s.node), chanWathInfo, util.NewTickler())
|
||||
ds.GetMetaCache()
|
||||
s.Require().NoError(err)
|
||||
s.node.flowgraphManager.AddFlowgraph(ds)
|
||||
|
||||
fgservice, ok := s.node.flowgraphManager.GetFlowgraphService(dmChannelName)
|
||||
s.Require().True(ok)
|
||||
|
||||
fgservice.GetMetaCache().AddSegment(&datapb.SegmentInfo{
|
||||
ID: segmentID,
|
||||
CollectionID: 1,
|
||||
PartitionID: 2,
|
||||
@ -419,7 +428,7 @@ func (s *DataNodeServicesSuite) TestShowConfigurations() {
|
||||
func (s *DataNodeServicesSuite) TestGetMetrics() {
|
||||
node := &DataNode{}
|
||||
node.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}})
|
||||
node.flowgraphManager = newFlowgraphManager()
|
||||
node.flowgraphManager = pipeline.NewFlowgraphManager()
|
||||
// server is closed
|
||||
node.stateCode.Store(commonpb.StateCode_Abnormal)
|
||||
resp, err := node.GetMetrics(s.ctx, &milvuspb.GetMetricsRequest{})
|
||||
@ -618,10 +627,9 @@ func (s *DataNodeServicesSuite) TestSyncSegments() {
|
||||
}, func(*datapb.SegmentInfo) *metacache.BloomFilterSet {
|
||||
return metacache.NewBloomFilterSet()
|
||||
})
|
||||
mockFlowgraphManager := NewMockFlowgraphManager(s.T())
|
||||
mockFlowgraphManager.EXPECT().GetFlowgraphService(mock.Anything).Return(&dataSyncService{
|
||||
metacache: cache,
|
||||
}, true)
|
||||
mockFlowgraphManager := pipeline.NewMockFlowgraphManager(s.T())
|
||||
mockFlowgraphManager.EXPECT().GetFlowgraphService(mock.Anything).
|
||||
Return(pipeline.NewDataSyncServiceWithMetaCache(cache), true)
|
||||
s.node.flowgraphManager = mockFlowgraphManager
|
||||
ctx := context.Background()
|
||||
req := &datapb.SyncSegmentsRequest{
|
||||
|
@ -117,7 +117,7 @@ type MockTask_HandleError_Call struct {
|
||||
}
|
||||
|
||||
// HandleError is a helper method to define mock.On call
|
||||
// - _a0 error
|
||||
// - _a0 error
|
||||
func (_e *MockTask_Expecter) HandleError(_a0 interface{}) *MockTask_HandleError_Call {
|
||||
return &MockTask_HandleError_Call{Call: _e.mock.On("HandleError", _a0)}
|
||||
}
|
||||
|
@ -14,9 +14,11 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package util
|
||||
|
||||
import "github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
import (
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
// Cache stores flushing segments' ids to prevent flushing the same segment again and again.
|
||||
//
|
||||
@ -29,8 +31,8 @@ type Cache struct {
|
||||
*typeutil.ConcurrentSet[UniqueID]
|
||||
}
|
||||
|
||||
// newCache returns a new Cache
|
||||
func newCache() *Cache {
|
||||
// NewCache returns a new Cache
|
||||
func NewCache() *Cache {
|
||||
return &Cache{
|
||||
ConcurrentSet: typeutil.NewConcurrentSet[UniqueID](),
|
||||
}
|
@ -14,16 +14,29 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package util
|
||||
|
||||
import (
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
)
|
||||
|
||||
func TestMain(t *testing.M) {
|
||||
paramtable.Init()
|
||||
err := InitGlobalRateCollector()
|
||||
if err != nil {
|
||||
panic("init test failed, err = " + err.Error())
|
||||
}
|
||||
code := t.Run()
|
||||
os.Exit(code)
|
||||
}
|
||||
|
||||
func TestSegmentCache(t *testing.T) {
|
||||
segCache := newCache()
|
||||
segCache := NewCache()
|
||||
|
||||
assert.False(t, segCache.checkIfCached(0))
|
||||
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package util
|
||||
|
||||
import (
|
||||
"context"
|
||||
@ -41,7 +41,7 @@ type channelCPUpdateTask struct {
|
||||
flush bool // indicates whether the task originates from flush
|
||||
}
|
||||
|
||||
type channelCheckpointUpdater struct {
|
||||
type ChannelCheckpointUpdater struct {
|
||||
broker broker.Broker
|
||||
|
||||
mu sync.RWMutex
|
||||
@ -52,8 +52,8 @@ type channelCheckpointUpdater struct {
|
||||
closeOnce sync.Once
|
||||
}
|
||||
|
||||
func newChannelCheckpointUpdater(broker broker.Broker) *channelCheckpointUpdater {
|
||||
return &channelCheckpointUpdater{
|
||||
func NewChannelCheckpointUpdater(broker broker.Broker) *ChannelCheckpointUpdater {
|
||||
return &ChannelCheckpointUpdater{
|
||||
broker: broker,
|
||||
tasks: make(map[string]*channelCPUpdateTask),
|
||||
closeCh: make(chan struct{}),
|
||||
@ -61,7 +61,7 @@ func newChannelCheckpointUpdater(broker broker.Broker) *channelCheckpointUpdater
|
||||
}
|
||||
}
|
||||
|
||||
func (ccu *channelCheckpointUpdater) start() {
|
||||
func (ccu *ChannelCheckpointUpdater) Start() {
|
||||
log.Info("channel checkpoint updater start")
|
||||
ticker := time.NewTicker(paramtable.Get().DataNodeCfg.ChannelCheckpointUpdateTickInSeconds.GetAsDuration(time.Second))
|
||||
defer ticker.Stop()
|
||||
@ -90,21 +90,21 @@ func (ccu *channelCheckpointUpdater) start() {
|
||||
}
|
||||
}
|
||||
|
||||
func (ccu *channelCheckpointUpdater) getTask(channel string) (*channelCPUpdateTask, bool) {
|
||||
func (ccu *ChannelCheckpointUpdater) getTask(channel string) (*channelCPUpdateTask, bool) {
|
||||
ccu.mu.RLock()
|
||||
defer ccu.mu.RUnlock()
|
||||
task, ok := ccu.tasks[channel]
|
||||
return task, ok
|
||||
}
|
||||
|
||||
func (ccu *channelCheckpointUpdater) trigger() {
|
||||
func (ccu *ChannelCheckpointUpdater) trigger() {
|
||||
select {
|
||||
case ccu.notifyChan <- struct{}{}:
|
||||
default:
|
||||
}
|
||||
}
|
||||
|
||||
func (ccu *channelCheckpointUpdater) updateCheckpoints(tasks []*channelCPUpdateTask) {
|
||||
func (ccu *ChannelCheckpointUpdater) updateCheckpoints(tasks []*channelCPUpdateTask) {
|
||||
taskGroups := lo.Chunk(tasks, paramtable.Get().DataNodeCfg.MaxChannelCheckpointsPerRPC.GetAsInt())
|
||||
updateChanCPMaxParallel := paramtable.Get().DataNodeCfg.UpdateChannelCheckpointMaxParallel.GetAsInt()
|
||||
if updateChanCPMaxParallel <= 0 {
|
||||
@ -152,7 +152,7 @@ func (ccu *channelCheckpointUpdater) updateCheckpoints(tasks []*channelCPUpdateT
|
||||
})
|
||||
}
|
||||
|
||||
func (ccu *channelCheckpointUpdater) execute() {
|
||||
func (ccu *ChannelCheckpointUpdater) execute() {
|
||||
ccu.mu.RLock()
|
||||
tasks := lo.Values(ccu.tasks)
|
||||
ccu.mu.RUnlock()
|
||||
@ -160,7 +160,7 @@ func (ccu *channelCheckpointUpdater) execute() {
|
||||
ccu.updateCheckpoints(tasks)
|
||||
}
|
||||
|
||||
func (ccu *channelCheckpointUpdater) AddTask(channelPos *msgpb.MsgPosition, flush bool, callback func()) {
|
||||
func (ccu *ChannelCheckpointUpdater) AddTask(channelPos *msgpb.MsgPosition, flush bool, callback func()) {
|
||||
if channelPos == nil || channelPos.GetMsgID() == nil || channelPos.GetChannelName() == "" {
|
||||
log.Warn("illegal checkpoint", zap.Any("pos", channelPos))
|
||||
return
|
||||
@ -201,13 +201,13 @@ func (ccu *channelCheckpointUpdater) AddTask(channelPos *msgpb.MsgPosition, flus
|
||||
}
|
||||
}
|
||||
|
||||
func (ccu *channelCheckpointUpdater) taskNum() int {
|
||||
func (ccu *ChannelCheckpointUpdater) taskNum() int {
|
||||
ccu.mu.RLock()
|
||||
defer ccu.mu.RUnlock()
|
||||
return len(ccu.tasks)
|
||||
}
|
||||
|
||||
func (ccu *channelCheckpointUpdater) close() {
|
||||
func (ccu *ChannelCheckpointUpdater) Close() {
|
||||
ccu.closeOnce.Do(func() {
|
||||
close(ccu.closeCh)
|
||||
})
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package util
|
||||
|
||||
import (
|
||||
"context"
|
||||
@ -36,12 +36,12 @@ type ChannelCPUpdaterSuite struct {
|
||||
suite.Suite
|
||||
|
||||
broker *broker.MockBroker
|
||||
updater *channelCheckpointUpdater
|
||||
updater *ChannelCheckpointUpdater
|
||||
}
|
||||
|
||||
func (s *ChannelCPUpdaterSuite) SetupTest() {
|
||||
s.broker = broker.NewMockBroker(s.T())
|
||||
s.updater = newChannelCheckpointUpdater(s.broker)
|
||||
s.updater = NewChannelCheckpointUpdater(s.broker)
|
||||
}
|
||||
|
||||
func (s *ChannelCPUpdaterSuite) TestUpdate() {
|
||||
@ -53,8 +53,8 @@ func (s *ChannelCPUpdaterSuite) TestUpdate() {
|
||||
return nil
|
||||
})
|
||||
|
||||
go s.updater.start()
|
||||
defer s.updater.close()
|
||||
go s.updater.Start()
|
||||
defer s.updater.Close()
|
||||
|
||||
tasksNum := 100000
|
||||
counter := atomic.NewInt64(0)
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package util
|
||||
|
||||
import (
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
@ -22,8 +22,8 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
||||
)
|
||||
|
||||
// reviseVChannelInfo will revise the datapb.VchannelInfo for upgrade compatibility from 2.0.2
|
||||
func reviseVChannelInfo(vChannel *datapb.VchannelInfo) {
|
||||
// ReviseVChannelInfo will revise the datapb.VchannelInfo for upgrade compatibility from 2.0.2
|
||||
func ReviseVChannelInfo(vChannel *datapb.VchannelInfo) {
|
||||
removeDuplicateSegmentIDFn := func(ids []int64) []int64 {
|
||||
result := make([]int64, 0, len(ids))
|
||||
existDict := make(map[int64]bool)
|
@ -14,65 +14,76 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package util
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/pkg/util/ratelimitutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
// rateCol is global rateCollector in DataNode.
|
||||
// RateCol is global RateCollector in DataNode.
|
||||
var (
|
||||
rateCol *rateCollector
|
||||
RateCol *RateCollector
|
||||
initOnce sync.Once
|
||||
)
|
||||
|
||||
// rateCollector helps to collect and calculate values (like rate, timeTick and etc...).
|
||||
type rateCollector struct {
|
||||
// RateCollector helps to collect and calculate values (like rate, timeTick and etc...).
|
||||
type RateCollector struct {
|
||||
*ratelimitutil.RateCollector
|
||||
|
||||
flowGraphTtMu sync.Mutex
|
||||
flowGraphTt map[string]Timestamp
|
||||
}
|
||||
|
||||
func initGlobalRateCollector() error {
|
||||
func InitGlobalRateCollector() error {
|
||||
var err error
|
||||
initOnce.Do(func() {
|
||||
rateCol, err = newRateCollector()
|
||||
RateCol, err = NewRateCollector()
|
||||
})
|
||||
RateCol.Register(metricsinfo.InsertConsumeThroughput)
|
||||
RateCol.Register(metricsinfo.DeleteConsumeThroughput)
|
||||
return err
|
||||
}
|
||||
|
||||
// newRateCollector returns a new rateCollector.
|
||||
func newRateCollector() (*rateCollector, error) {
|
||||
func DeregisterRateCollector(label string) {
|
||||
RateCol.Deregister(label)
|
||||
}
|
||||
|
||||
func RegisterRateCollector(label string) {
|
||||
RateCol.Register(label)
|
||||
}
|
||||
|
||||
// newRateCollector returns a new RateCollector.
|
||||
func NewRateCollector() (*RateCollector, error) {
|
||||
rc, err := ratelimitutil.NewRateCollector(ratelimitutil.DefaultWindow, ratelimitutil.DefaultGranularity, false)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &rateCollector{
|
||||
return &RateCollector{
|
||||
RateCollector: rc,
|
||||
flowGraphTt: make(map[string]Timestamp),
|
||||
}, nil
|
||||
}
|
||||
|
||||
// updateFlowGraphTt updates rateCollector's flow graph time tick.
|
||||
func (r *rateCollector) updateFlowGraphTt(channel string, t Timestamp) {
|
||||
// UpdateFlowGraphTt updates RateCollector's flow graph time tick.
|
||||
func (r *RateCollector) UpdateFlowGraphTt(channel string, t Timestamp) {
|
||||
r.flowGraphTtMu.Lock()
|
||||
defer r.flowGraphTtMu.Unlock()
|
||||
r.flowGraphTt[channel] = t
|
||||
}
|
||||
|
||||
// removeFlowGraphChannel removes channel from flowGraphTt.
|
||||
func (r *rateCollector) removeFlowGraphChannel(channel string) {
|
||||
// RemoveFlowGraphChannel removes channel from flowGraphTt.
|
||||
func (r *RateCollector) RemoveFlowGraphChannel(channel string) {
|
||||
r.flowGraphTtMu.Lock()
|
||||
defer r.flowGraphTtMu.Unlock()
|
||||
delete(r.flowGraphTt, channel)
|
||||
}
|
||||
|
||||
// getMinFlowGraphTt returns the vchannel and minimal time tick of flow graphs.
|
||||
func (r *rateCollector) getMinFlowGraphTt() (string, Timestamp) {
|
||||
// GetMinFlowGraphTt returns the vchannel and minimal time tick of flow graphs.
|
||||
func (r *RateCollector) GetMinFlowGraphTt() (string, Timestamp) {
|
||||
r.flowGraphTtMu.Lock()
|
||||
defer r.flowGraphTtMu.Unlock()
|
||||
minTt := typeutil.MaxTimestamp
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package util
|
||||
|
||||
import (
|
||||
"testing"
|
||||
@ -26,16 +26,16 @@ import (
|
||||
|
||||
func TestRateCollector(t *testing.T) {
|
||||
t.Run("test FlowGraphTt", func(t *testing.T) {
|
||||
collector, err := newRateCollector()
|
||||
collector, err := NewRateCollector()
|
||||
assert.NoError(t, err)
|
||||
|
||||
c, minTt := collector.getMinFlowGraphTt()
|
||||
c, minTt := collector.GetMinFlowGraphTt()
|
||||
assert.Equal(t, "", c)
|
||||
assert.Equal(t, typeutil.MaxTimestamp, minTt)
|
||||
collector.updateFlowGraphTt("channel1", 100)
|
||||
collector.updateFlowGraphTt("channel2", 200)
|
||||
collector.updateFlowGraphTt("channel3", 50)
|
||||
c, minTt = collector.getMinFlowGraphTt()
|
||||
collector.UpdateFlowGraphTt("channel1", 100)
|
||||
collector.UpdateFlowGraphTt("channel2", 200)
|
||||
collector.UpdateFlowGraphTt("channel3", 50)
|
||||
c, minTt = collector.GetMinFlowGraphTt()
|
||||
assert.Equal(t, "channel3", c)
|
||||
assert.Equal(t, Timestamp(50), minTt)
|
||||
})
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package util
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
@ -22,12 +22,8 @@ import (
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"math"
|
||||
"math/rand"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/samber/lo"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/grpc"
|
||||
|
||||
@ -35,31 +31,22 @@ import (
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/datanode/broker"
|
||||
"github.com/milvus-io/milvus/internal/datanode/metacache"
|
||||
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
|
||||
"github.com/milvus-io/milvus/internal/datanode/writebuffer"
|
||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/etcdpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/kv"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/pkg/util/etcd"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
|
||||
const ctxTimeInMillisecond = 5000
|
||||
const returnError = "ReturnError"
|
||||
|
||||
type ctxKey struct{}
|
||||
|
||||
// As used in data_sync_service_test.go
|
||||
var segID2SegInfo = map[int64]*datapb.SegmentInfo{
|
||||
@ -81,79 +68,6 @@ var segID2SegInfo = map[int64]*datapb.SegmentInfo{
|
||||
},
|
||||
}
|
||||
|
||||
func newIDLEDataNodeMock(ctx context.Context, pkType schemapb.DataType) *DataNode {
|
||||
factory := dependency.NewDefaultFactory(true)
|
||||
node := NewDataNode(ctx, factory)
|
||||
node.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}})
|
||||
node.dispClient = msgdispatcher.NewClient(factory, typeutil.DataNodeRole, paramtable.GetNodeID())
|
||||
|
||||
broker := &broker.MockBroker{}
|
||||
broker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
broker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).Return([]*datapb.SegmentInfo{}, nil).Maybe()
|
||||
|
||||
node.broker = broker
|
||||
node.timeTickSender = newTimeTickSender(node.broker, 0)
|
||||
|
||||
syncMgr, _ := syncmgr.NewSyncManager(node.chunkManager, node.allocator)
|
||||
|
||||
node.syncMgr = syncMgr
|
||||
node.writeBufferManager = writebuffer.NewManager(node.syncMgr)
|
||||
|
||||
return node
|
||||
}
|
||||
|
||||
func newTestEtcdKV() (kv.WatchKV, error) {
|
||||
etcdCli, err := etcd.GetEtcdClient(
|
||||
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
|
||||
Params.EtcdCfg.EtcdUseSSL.GetAsBool(),
|
||||
Params.EtcdCfg.Endpoints.GetAsStrings(),
|
||||
Params.EtcdCfg.EtcdTLSCert.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSKey.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSCACert.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSMinVersion.GetValue())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue()), nil
|
||||
}
|
||||
|
||||
func clearEtcd(rootPath string) error {
|
||||
client, err := etcd.GetEtcdClient(
|
||||
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
|
||||
Params.EtcdCfg.EtcdUseSSL.GetAsBool(),
|
||||
Params.EtcdCfg.Endpoints.GetAsStrings(),
|
||||
Params.EtcdCfg.EtcdTLSCert.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSKey.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSCACert.GetValue(),
|
||||
Params.EtcdCfg.EtcdTLSMinVersion.GetValue())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
etcdKV := etcdkv.NewEtcdKV(client, rootPath)
|
||||
|
||||
err = etcdKV.RemoveWithPrefix("writer/segment")
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
_, _, err = etcdKV.LoadWithPrefix("writer/segment")
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
log.Debug("Clear ETCD with prefix writer/segment ")
|
||||
|
||||
err = etcdKV.RemoveWithPrefix("writer/ddl")
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
_, _, err = etcdKV.LoadWithPrefix("writer/ddl")
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
log.Debug("Clear ETCD with prefix writer/ddl")
|
||||
return nil
|
||||
}
|
||||
|
||||
type MetaFactory struct{}
|
||||
|
||||
func NewMetaFactory() *MetaFactory {
|
||||
@ -877,66 +791,6 @@ func (df *DataFactory) GenMsgStreamDeleteMsgWithTs(idx int, pks []storage.Primar
|
||||
return msg
|
||||
}
|
||||
|
||||
func genFlowGraphInsertMsg(chanName string) flowGraphMsg {
|
||||
timeRange := TimeRange{
|
||||
timestampMin: 0,
|
||||
timestampMax: math.MaxUint64,
|
||||
}
|
||||
|
||||
startPos := []*msgpb.MsgPosition{
|
||||
{
|
||||
ChannelName: chanName,
|
||||
MsgID: make([]byte, 0),
|
||||
Timestamp: tsoutil.ComposeTSByTime(time.Now(), 0),
|
||||
},
|
||||
}
|
||||
|
||||
fgMsg := &flowGraphMsg{
|
||||
insertMessages: make([]*msgstream.InsertMsg, 0),
|
||||
timeRange: TimeRange{
|
||||
timestampMin: timeRange.timestampMin,
|
||||
timestampMax: timeRange.timestampMax,
|
||||
},
|
||||
startPositions: startPos,
|
||||
endPositions: startPos,
|
||||
}
|
||||
|
||||
dataFactory := NewDataFactory()
|
||||
fgMsg.insertMessages = append(fgMsg.insertMessages, dataFactory.GetMsgStreamInsertMsgs(2)...)
|
||||
|
||||
return *fgMsg
|
||||
}
|
||||
|
||||
func genFlowGraphDeleteMsg(pks []storage.PrimaryKey, chanName string) flowGraphMsg {
|
||||
timeRange := TimeRange{
|
||||
timestampMin: 0,
|
||||
timestampMax: math.MaxUint64,
|
||||
}
|
||||
|
||||
startPos := []*msgpb.MsgPosition{
|
||||
{
|
||||
ChannelName: chanName,
|
||||
MsgID: make([]byte, 0),
|
||||
Timestamp: 0,
|
||||
},
|
||||
}
|
||||
|
||||
fgMsg := &flowGraphMsg{
|
||||
insertMessages: make([]*msgstream.InsertMsg, 0),
|
||||
timeRange: TimeRange{
|
||||
timestampMin: timeRange.timestampMin,
|
||||
timestampMax: timeRange.timestampMax,
|
||||
},
|
||||
startPositions: startPos,
|
||||
endPositions: startPos,
|
||||
}
|
||||
|
||||
dataFactory := NewDataFactory()
|
||||
fgMsg.deleteMessages = append(fgMsg.deleteMessages, dataFactory.GenMsgStreamDeleteMsg(pks, chanName))
|
||||
|
||||
return *fgMsg
|
||||
}
|
||||
|
||||
func (m *RootCoordFactory) setCollectionID(id UniqueID) {
|
||||
m.collectionID = id
|
||||
}
|
||||
@ -1065,139 +919,12 @@ func (f *FailMessageStreamFactory) NewTtMsgStream(ctx context.Context) (msgstrea
|
||||
return nil, errors.New("mocked failure")
|
||||
}
|
||||
|
||||
func genInsertDataWithPKs(PKs [2]storage.PrimaryKey, dataType schemapb.DataType) *InsertData {
|
||||
iD := genInsertData(2)
|
||||
switch dataType {
|
||||
case schemapb.DataType_Int64:
|
||||
values := make([]int64, len(PKs))
|
||||
for index, pk := range PKs {
|
||||
values[index] = pk.(*storage.Int64PrimaryKey).Value
|
||||
}
|
||||
iD.Data[106].(*storage.Int64FieldData).Data = values
|
||||
case schemapb.DataType_VarChar:
|
||||
values := make([]string, len(PKs))
|
||||
for index, pk := range PKs {
|
||||
values[index] = pk.(*storage.VarCharPrimaryKey).Value
|
||||
}
|
||||
iD.Data[109].(*storage.StringFieldData).Data = values
|
||||
default:
|
||||
// TODO::
|
||||
}
|
||||
return iD
|
||||
}
|
||||
|
||||
func genTestStat(meta *etcdpb.CollectionMeta) *storage.PrimaryKeyStats {
|
||||
var pkFieldID, pkFieldType int64
|
||||
for _, field := range meta.Schema.Fields {
|
||||
if field.IsPrimaryKey {
|
||||
pkFieldID = field.FieldID
|
||||
pkFieldType = int64(field.DataType)
|
||||
}
|
||||
}
|
||||
stats, _ := storage.NewPrimaryKeyStats(pkFieldID, pkFieldType, 100)
|
||||
return stats
|
||||
}
|
||||
|
||||
func genInsertData(rowNum int) *InsertData {
|
||||
return &InsertData{
|
||||
Data: map[int64]storage.FieldData{
|
||||
0: &storage.Int64FieldData{
|
||||
Data: lo.RepeatBy(rowNum, func(i int) int64 { return int64(i + 1) }),
|
||||
},
|
||||
1: &storage.Int64FieldData{
|
||||
Data: lo.RepeatBy(rowNum, func(i int) int64 { return int64(i + 3) }),
|
||||
},
|
||||
100: &storage.FloatVectorFieldData{
|
||||
Data: lo.RepeatBy(rowNum*2, func(i int) float32 { return rand.Float32() }),
|
||||
Dim: 2,
|
||||
},
|
||||
101: &storage.BinaryVectorFieldData{
|
||||
Data: lo.RepeatBy(rowNum*4, func(i int) byte { return byte(rand.Intn(256)) }),
|
||||
Dim: 32,
|
||||
},
|
||||
102: &storage.BoolFieldData{
|
||||
Data: lo.RepeatBy(rowNum, func(i int) bool { return i%2 == 0 }),
|
||||
},
|
||||
103: &storage.Int8FieldData{
|
||||
Data: lo.RepeatBy(rowNum, func(i int) int8 { return int8(i) }),
|
||||
},
|
||||
104: &storage.Int16FieldData{
|
||||
Data: lo.RepeatBy(rowNum, func(i int) int16 { return int16(i) }),
|
||||
},
|
||||
105: &storage.Int32FieldData{
|
||||
Data: lo.RepeatBy(rowNum, func(i int) int32 { return int32(i) }),
|
||||
},
|
||||
106: &storage.Int64FieldData{
|
||||
Data: lo.RepeatBy(rowNum, func(i int) int64 { return int64(i) }),
|
||||
},
|
||||
107: &storage.FloatFieldData{
|
||||
Data: lo.RepeatBy(rowNum, func(i int) float32 { return rand.Float32() }),
|
||||
},
|
||||
108: &storage.DoubleFieldData{
|
||||
Data: lo.RepeatBy(rowNum, func(i int) float64 { return rand.Float64() }),
|
||||
},
|
||||
109: &storage.StringFieldData{
|
||||
Data: lo.RepeatBy(rowNum, func(i int) string { return fmt.Sprintf("test%d", i) }),
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func genEmptyInsertData() *InsertData {
|
||||
return &InsertData{
|
||||
Data: map[int64]storage.FieldData{
|
||||
0: &storage.Int64FieldData{
|
||||
Data: []int64{},
|
||||
},
|
||||
1: &storage.Int64FieldData{
|
||||
Data: []int64{},
|
||||
},
|
||||
100: &storage.FloatVectorFieldData{
|
||||
Data: []float32{},
|
||||
Dim: 2,
|
||||
},
|
||||
101: &storage.BinaryVectorFieldData{
|
||||
Data: []byte{},
|
||||
Dim: 32,
|
||||
},
|
||||
102: &storage.BoolFieldData{
|
||||
Data: []bool{},
|
||||
},
|
||||
103: &storage.Int8FieldData{
|
||||
Data: []int8{},
|
||||
},
|
||||
104: &storage.Int16FieldData{
|
||||
Data: []int16{},
|
||||
},
|
||||
105: &storage.Int32FieldData{
|
||||
Data: []int32{},
|
||||
},
|
||||
106: &storage.Int64FieldData{
|
||||
Data: []int64{},
|
||||
},
|
||||
107: &storage.FloatFieldData{
|
||||
Data: []float32{},
|
||||
},
|
||||
108: &storage.DoubleFieldData{
|
||||
Data: []float64{},
|
||||
},
|
||||
109: &storage.StringFieldData{
|
||||
Data: []string{},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func genTestTickler() *etcdTickler {
|
||||
return newEtcdTickler(0, "", nil, nil, 0)
|
||||
}
|
||||
|
||||
// MockDataSuiteBase compose some mock dependency to generate test dataset
|
||||
type MockDataSuiteBase struct {
|
||||
schema *schemapb.CollectionSchema
|
||||
}
|
||||
|
||||
func (s *MockDataSuiteBase) prepareData() {
|
||||
func (s *MockDataSuiteBase) PrepareData() {
|
||||
s.schema = &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
@ -1214,3 +941,34 @@ func (s *MockDataSuiteBase) prepareData() {
|
||||
func EmptyBfsFactory(info *datapb.SegmentInfo) *metacache.BloomFilterSet {
|
||||
return metacache.NewBloomFilterSet()
|
||||
}
|
||||
|
||||
func GetWatchInfoByOpID(opID UniqueID, channel string, state datapb.ChannelWatchState) *datapb.ChannelWatchInfo {
|
||||
return &datapb.ChannelWatchInfo{
|
||||
OpID: opID,
|
||||
State: state,
|
||||
Vchan: &datapb.VchannelInfo{
|
||||
CollectionID: 1,
|
||||
ChannelName: channel,
|
||||
},
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Name: "test_collection",
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: common.DimKey, Value: "128"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
51
internal/datanode/util/tickler.go
Normal file
51
internal/datanode/util/tickler.go
Normal file
@ -0,0 +1,51 @@
|
||||
package util
|
||||
|
||||
import "go.uber.org/atomic"
|
||||
|
||||
// Tickler counts every time when called inc(),
|
||||
type Tickler struct {
|
||||
count *atomic.Int32
|
||||
total *atomic.Int32
|
||||
closedSig *atomic.Bool
|
||||
|
||||
progressSig chan struct{}
|
||||
}
|
||||
|
||||
func (t *Tickler) Inc() {
|
||||
t.count.Inc()
|
||||
t.progressSig <- struct{}{}
|
||||
}
|
||||
|
||||
func (t *Tickler) SetTotal(total int32) {
|
||||
t.total.Store(total)
|
||||
}
|
||||
|
||||
// progress returns the count over total if total is set
|
||||
// else just return the count number.
|
||||
func (t *Tickler) Progress() int32 {
|
||||
if t.total.Load() == 0 {
|
||||
return t.count.Load()
|
||||
}
|
||||
return (t.count.Load() / t.total.Load()) * 100
|
||||
}
|
||||
|
||||
func (t *Tickler) Close() {
|
||||
t.closedSig.CompareAndSwap(false, true)
|
||||
}
|
||||
|
||||
func (t *Tickler) IsClosed() bool {
|
||||
return t.closedSig.Load()
|
||||
}
|
||||
|
||||
func (t *Tickler) GetProgressSig() chan struct{} {
|
||||
return t.progressSig
|
||||
}
|
||||
|
||||
func NewTickler() *Tickler {
|
||||
return &Tickler{
|
||||
count: atomic.NewInt32(0),
|
||||
total: atomic.NewInt32(0),
|
||||
closedSig: atomic.NewBool(false),
|
||||
progressSig: make(chan struct{}, 200),
|
||||
}
|
||||
}
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package util
|
||||
|
||||
import (
|
||||
"context"
|
||||
@ -29,13 +29,14 @@ import (
|
||||
"github.com/milvus-io/milvus/internal/datanode/broker"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/retry"
|
||||
)
|
||||
|
||||
// timeTickSender is to merge channel states updated by flow graph node and send to datacoord periodically
|
||||
// timeTickSender hold segmentStats cache for each channel,
|
||||
// TimeTickSender is to merge channel states updated by flow graph node and send to datacoord periodically
|
||||
// TimeTickSender hold segmentStats cache for each channel,
|
||||
// after send succeeds will clean the cache earlier than last sent timestamp
|
||||
type timeTickSender struct {
|
||||
type TimeTickSender struct {
|
||||
nodeID int64
|
||||
broker broker.Broker
|
||||
|
||||
@ -53,22 +54,23 @@ type channelStats struct {
|
||||
lastTs uint64
|
||||
}
|
||||
|
||||
// data struct only used in timeTickSender
|
||||
// data struct only used in TimeTickSender
|
||||
type segmentStats struct {
|
||||
*commonpb.SegmentStats
|
||||
ts uint64
|
||||
}
|
||||
|
||||
func newTimeTickSender(broker broker.Broker, nodeID int64, opts ...retry.Option) *timeTickSender {
|
||||
return &timeTickSender{
|
||||
func NewTimeTickSender(broker broker.Broker, nodeID int64, opts ...retry.Option) *TimeTickSender {
|
||||
return &TimeTickSender{
|
||||
nodeID: nodeID,
|
||||
broker: broker,
|
||||
statsCache: make(map[string]*channelStats),
|
||||
options: opts,
|
||||
mu: sync.RWMutex{},
|
||||
}
|
||||
}
|
||||
|
||||
func (m *timeTickSender) start() {
|
||||
func (m *TimeTickSender) Start() {
|
||||
m.wg.Add(1)
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
m.cancelFunc = cancel
|
||||
@ -76,22 +78,23 @@ func (m *timeTickSender) start() {
|
||||
defer m.wg.Done()
|
||||
m.work(ctx)
|
||||
}()
|
||||
log.Info("timeTick sender started")
|
||||
}
|
||||
|
||||
func (m *timeTickSender) Stop() {
|
||||
func (m *TimeTickSender) Stop() {
|
||||
if m.cancelFunc != nil {
|
||||
m.cancelFunc()
|
||||
m.wg.Wait()
|
||||
}
|
||||
}
|
||||
|
||||
func (m *timeTickSender) work(ctx context.Context) {
|
||||
ticker := time.NewTicker(Params.DataNodeCfg.DataNodeTimeTickInterval.GetAsDuration(time.Millisecond))
|
||||
func (m *TimeTickSender) work(ctx context.Context) {
|
||||
ticker := time.NewTicker(paramtable.Get().DataNodeCfg.DataNodeTimeTickInterval.GetAsDuration(time.Millisecond))
|
||||
defer ticker.Stop()
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
log.Info("timeTickSender context done")
|
||||
log.Info("TimeTickSender context done")
|
||||
return
|
||||
case <-ticker.C:
|
||||
m.sendReport(ctx)
|
||||
@ -99,7 +102,7 @@ func (m *timeTickSender) work(ctx context.Context) {
|
||||
}
|
||||
}
|
||||
|
||||
func (m *timeTickSender) update(channelName string, timestamp uint64, segStats []*commonpb.SegmentStats) {
|
||||
func (m *TimeTickSender) Update(channelName string, timestamp uint64, segStats []*commonpb.SegmentStats) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
_, ok := m.statsCache[channelName]
|
||||
@ -118,7 +121,7 @@ func (m *timeTickSender) update(channelName string, timestamp uint64, segStats [
|
||||
m.statsCache[channelName].lastTs = timestamp
|
||||
}
|
||||
|
||||
func (m *timeTickSender) assembleDatanodeTtMsg() ([]*msgpb.DataNodeTtMsg, map[string]uint64) {
|
||||
func (m *TimeTickSender) assembleDatanodeTtMsg() ([]*msgpb.DataNodeTtMsg, map[string]uint64) {
|
||||
m.mu.RLock()
|
||||
defer m.mu.RUnlock()
|
||||
|
||||
@ -144,7 +147,7 @@ func (m *timeTickSender) assembleDatanodeTtMsg() ([]*msgpb.DataNodeTtMsg, map[st
|
||||
return msgs, lastSentTss
|
||||
}
|
||||
|
||||
func (m *timeTickSender) cleanStatesCache(lastSentTss map[string]uint64) {
|
||||
func (m *TimeTickSender) cleanStatesCache(lastSentTss map[string]uint64) {
|
||||
m.mu.Lock()
|
||||
defer m.mu.Unlock()
|
||||
sizeBeforeClean := len(m.statsCache)
|
||||
@ -161,12 +164,12 @@ func (m *timeTickSender) cleanStatesCache(lastSentTss map[string]uint64) {
|
||||
delete(m.statsCache, channelName)
|
||||
}
|
||||
}
|
||||
log.RatedDebug(30, "timeTickSender stats", zap.Any("lastSentTss", lastSentTss), zap.Int("sizeBeforeClean", sizeBeforeClean), zap.Int("sizeAfterClean", len(m.statsCache)))
|
||||
log.RatedDebug(30, "TimeTickSender stats", zap.Any("lastSentTss", lastSentTss), zap.Int("sizeBeforeClean", sizeBeforeClean), zap.Int("sizeAfterClean", len(m.statsCache)))
|
||||
}
|
||||
|
||||
func (m *timeTickSender) sendReport(ctx context.Context) error {
|
||||
func (m *TimeTickSender) sendReport(ctx context.Context) error {
|
||||
toSendMsgs, sendLastTss := m.assembleDatanodeTtMsg()
|
||||
log.RatedDebug(30, "timeTickSender send datanode timetick message", zap.Any("toSendMsgs", toSendMsgs), zap.Any("sendLastTss", sendLastTss))
|
||||
log.RatedDebug(30, "TimeTickSender send datanode timetick message", zap.Any("toSendMsgs", toSendMsgs), zap.Any("sendLastTss", sendLastTss))
|
||||
err := retry.Do(ctx, func() error {
|
||||
return m.broker.ReportTimeTick(ctx, toSendMsgs)
|
||||
}, m.options...)
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package util
|
||||
|
||||
import (
|
||||
"context"
|
||||
@ -40,7 +40,7 @@ func TestTimetickManagerNormal(t *testing.T) {
|
||||
broker := broker.NewMockBroker(t)
|
||||
broker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
|
||||
manager := newTimeTickSender(broker, 0)
|
||||
manager := NewTimeTickSender(broker, 0)
|
||||
|
||||
channelName1 := "channel1"
|
||||
ts := uint64(time.Now().UnixMilli())
|
||||
@ -53,7 +53,7 @@ func TestTimetickManagerNormal(t *testing.T) {
|
||||
},
|
||||
}
|
||||
// update first time
|
||||
manager.update(channelName1, ts, segmentStats)
|
||||
manager.Update(channelName1, ts, segmentStats)
|
||||
|
||||
chanStats, exist := manager.statsCache[channelName1]
|
||||
assert.Equal(t, true, exist)
|
||||
@ -76,7 +76,7 @@ func TestTimetickManagerNormal(t *testing.T) {
|
||||
},
|
||||
}
|
||||
ts2 := ts + 100
|
||||
manager.update(channelName1, ts2, segmentStats2)
|
||||
manager.Update(channelName1, ts2, segmentStats2)
|
||||
|
||||
chanStats, exist = manager.statsCache[channelName1]
|
||||
assert.Equal(t, true, exist)
|
||||
@ -106,7 +106,7 @@ func TestTimetickManagerNormal(t *testing.T) {
|
||||
NumRows: 3333300,
|
||||
},
|
||||
}
|
||||
manager.update(channelName2, ts3, segmentStats3)
|
||||
manager.Update(channelName2, ts3, segmentStats3)
|
||||
|
||||
err := manager.sendReport(ctx)
|
||||
assert.NoError(t, err)
|
||||
@ -130,7 +130,7 @@ func TestTimetickManagerNormal(t *testing.T) {
|
||||
NumRows: 3333300,
|
||||
},
|
||||
}
|
||||
manager.update(channelName3, ts4, segmentStats4)
|
||||
manager.Update(channelName3, ts4, segmentStats4)
|
||||
|
||||
err = manager.sendReport(ctx)
|
||||
assert.NoError(t, err)
|
||||
@ -145,7 +145,7 @@ func TestTimetickManagerSendErr(t *testing.T) {
|
||||
broker := broker.NewMockBroker(t)
|
||||
broker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(errors.New("mock")).Maybe()
|
||||
|
||||
manager := newTimeTickSender(broker, 0, retry.Attempts(1))
|
||||
manager := NewTimeTickSender(broker, 0, retry.Attempts(1))
|
||||
|
||||
channelName1 := "channel1"
|
||||
ts := uint64(time.Now().Unix())
|
||||
@ -157,7 +157,7 @@ func TestTimetickManagerSendErr(t *testing.T) {
|
||||
},
|
||||
}
|
||||
// update first time
|
||||
manager.update(channelName1, ts, segmentStats)
|
||||
manager.Update(channelName1, ts, segmentStats)
|
||||
err := manager.sendReport(ctx)
|
||||
assert.Error(t, err)
|
||||
}
|
||||
@ -174,8 +174,8 @@ func TestTimetickManagerSendReport(t *testing.T) {
|
||||
}).
|
||||
Return(nil)
|
||||
mockDataCoord.EXPECT().ReportDataNodeTtMsgs(mock.Anything, mock.Anything).Return(merr.Status(nil), nil).Maybe()
|
||||
manager := newTimeTickSender(broker, 0)
|
||||
manager.start()
|
||||
manager := NewTimeTickSender(broker, 0)
|
||||
manager.Start()
|
||||
|
||||
assert.Eventually(t, func() bool {
|
||||
return called.Load()
|
@ -14,7 +14,7 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package datanode
|
||||
package util
|
||||
|
||||
import (
|
||||
"context"
|
||||
@ -22,6 +22,15 @@ import (
|
||||
"go.opentelemetry.io/otel"
|
||||
"go.opentelemetry.io/otel/trace"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/broker"
|
||||
"github.com/milvus-io/milvus/internal/datanode/compaction"
|
||||
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
|
||||
"github.com/milvus-io/milvus/internal/datanode/writebuffer"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/internal/util/dependency"
|
||||
"github.com/milvus-io/milvus/internal/util/sessionutil"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
||||
"github.com/milvus-io/milvus/pkg/mq/msgstream"
|
||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||
)
|
||||
@ -40,13 +49,28 @@ type (
|
||||
DSL = string
|
||||
)
|
||||
|
||||
// TimeRange is a range of timestamp contains the min-timestamp and max-timestamp
|
||||
type TimeRange struct {
|
||||
timestampMin Timestamp
|
||||
timestampMax Timestamp
|
||||
type PipelineParams struct {
|
||||
Ctx context.Context
|
||||
Broker broker.Broker
|
||||
SyncMgr syncmgr.SyncManager
|
||||
TimeTickSender *TimeTickSender // reference to TimeTickSender
|
||||
CompactionExecutor compaction.Executor // reference to compaction executor
|
||||
MsgStreamFactory dependency.Factory
|
||||
DispClient msgdispatcher.Client
|
||||
ChunkManager storage.ChunkManager
|
||||
Session *sessionutil.Session
|
||||
WriteBufferManager writebuffer.BufferManager
|
||||
CheckpointUpdater *ChannelCheckpointUpdater
|
||||
Allocator allocator.Allocator
|
||||
}
|
||||
|
||||
func startTracer(msg msgstream.TsMsg, name string) (context.Context, trace.Span) {
|
||||
// TimeRange is a range of timestamp contains the min-timestamp and max-timestamp
|
||||
type TimeRange struct {
|
||||
TimestampMin Timestamp
|
||||
TimestampMax Timestamp
|
||||
}
|
||||
|
||||
func StartTracer(msg msgstream.TsMsg, name string) (context.Context, trace.Span) {
|
||||
ctx := msg.TraceCtx()
|
||||
if ctx == nil {
|
||||
ctx = context.Background()
|
@ -2833,7 +2833,6 @@ user-task-polling:
|
||||
type dataCoordConfig struct {
|
||||
// --- CHANNEL ---
|
||||
WatchTimeoutInterval ParamItem `refreshable:"false"`
|
||||
EnableBalanceChannelWithRPC ParamItem `refreshable:"false"`
|
||||
LegacyVersionWithoutRPCWatch ParamItem `refreshable:"false"`
|
||||
ChannelBalanceSilentDuration ParamItem `refreshable:"true"`
|
||||
ChannelBalanceInterval ParamItem `refreshable:"true"`
|
||||
@ -2948,15 +2947,6 @@ func (p *dataCoordConfig) init(base *BaseTable) {
|
||||
}
|
||||
p.WatchTimeoutInterval.Init(base.mgr)
|
||||
|
||||
p.EnableBalanceChannelWithRPC = ParamItem{
|
||||
Key: "dataCoord.channel.balanceWithRpc",
|
||||
Version: "2.4.0",
|
||||
DefaultValue: "true",
|
||||
Doc: "Whether to enable balance with RPC, default to use etcd watch",
|
||||
Export: true,
|
||||
}
|
||||
p.EnableBalanceChannelWithRPC.Init(base.mgr)
|
||||
|
||||
p.LegacyVersionWithoutRPCWatch = ParamItem{
|
||||
Key: "dataCoord.channel.legacyVersionWithoutRPCWatch",
|
||||
Version: "2.4.1",
|
||||
|
@ -1,365 +0,0 @@
|
||||
// Licensed to the LF AI & Data foundation under one
|
||||
// or more contributor license agreements. See the NOTICE file
|
||||
// distributed with this work for additional information
|
||||
// regarding copyright ownership. The ASF licenses this file
|
||||
// to you under the Apache License, Version 2.0 (the
|
||||
// "License"); you may not use this file except in compliance
|
||||
// with the License. You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package watchcompatibility
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"strconv"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/stretchr/testify/suite"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
grpcdatacoord "github.com/milvus-io/milvus/internal/distributed/datacoord"
|
||||
grpcdatanode "github.com/milvus-io/milvus/internal/distributed/datanode"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/metric"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/tests/integration"
|
||||
)
|
||||
|
||||
type DataNodeCompatibility struct {
|
||||
integration.MiniClusterSuite
|
||||
maxGoRoutineNum int
|
||||
dim int
|
||||
numCollections int
|
||||
rowsPerCollection int
|
||||
waitTimeInSec time.Duration
|
||||
prefix string
|
||||
}
|
||||
|
||||
func (s *DataNodeCompatibility) setupParam() {
|
||||
s.maxGoRoutineNum = 100
|
||||
s.dim = 128
|
||||
s.numCollections = 1
|
||||
s.rowsPerCollection = 100
|
||||
s.waitTimeInSec = time.Second * 1
|
||||
}
|
||||
|
||||
func (s *DataNodeCompatibility) flush(collectionName string) {
|
||||
c := s.Cluster
|
||||
flushResp, err := c.Proxy.Flush(context.TODO(), &milvuspb.FlushRequest{
|
||||
DbName: "",
|
||||
CollectionNames: []string{collectionName},
|
||||
})
|
||||
s.NoError(err)
|
||||
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
|
||||
s.Require().True(has)
|
||||
s.Require().NotEmpty(segmentIDs)
|
||||
ids := segmentIDs.GetData()
|
||||
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
|
||||
s.True(has)
|
||||
|
||||
segments, err := c.MetaWatcher.ShowSegments()
|
||||
s.NoError(err)
|
||||
s.NotEmpty(segments)
|
||||
s.WaitForFlush(context.TODO(), ids, flushTs, "", collectionName)
|
||||
}
|
||||
|
||||
func (s *DataNodeCompatibility) loadCollection(collectionName string) {
|
||||
c := s.Cluster
|
||||
dbName := ""
|
||||
schema := integration.ConstructSchema(collectionName, s.dim, true)
|
||||
marshaledSchema, err := proto.Marshal(schema)
|
||||
s.NoError(err)
|
||||
|
||||
createCollectionStatus, err := c.Proxy.CreateCollection(context.TODO(), &milvuspb.CreateCollectionRequest{
|
||||
DbName: dbName,
|
||||
CollectionName: collectionName,
|
||||
Schema: marshaledSchema,
|
||||
ShardsNum: common.DefaultShardsNum,
|
||||
})
|
||||
s.NoError(err)
|
||||
|
||||
err = merr.Error(createCollectionStatus)
|
||||
s.NoError(err)
|
||||
|
||||
showCollectionsResp, err := c.Proxy.ShowCollections(context.TODO(), &milvuspb.ShowCollectionsRequest{})
|
||||
s.NoError(err)
|
||||
s.True(merr.Ok(showCollectionsResp.GetStatus()))
|
||||
|
||||
batchSize := 500000
|
||||
for start := 0; start < s.rowsPerCollection; start += batchSize {
|
||||
rowNum := batchSize
|
||||
if start+batchSize > s.rowsPerCollection {
|
||||
rowNum = s.rowsPerCollection - start
|
||||
}
|
||||
fVecColumn := integration.NewFloatVectorFieldData(integration.FloatVecField, rowNum, s.dim)
|
||||
hashKeys := integration.GenerateHashKeys(rowNum)
|
||||
insertResult, err := c.Proxy.Insert(context.TODO(), &milvuspb.InsertRequest{
|
||||
DbName: dbName,
|
||||
CollectionName: collectionName,
|
||||
FieldsData: []*schemapb.FieldData{fVecColumn},
|
||||
HashKeys: hashKeys,
|
||||
NumRows: uint32(rowNum),
|
||||
})
|
||||
s.NoError(err)
|
||||
s.True(merr.Ok(insertResult.GetStatus()))
|
||||
}
|
||||
s.flush(collectionName)
|
||||
|
||||
// create index
|
||||
createIndexStatus, err := c.Proxy.CreateIndex(context.TODO(), &milvuspb.CreateIndexRequest{
|
||||
CollectionName: collectionName,
|
||||
FieldName: integration.FloatVecField,
|
||||
IndexName: "_default",
|
||||
ExtraParams: integration.ConstructIndexParam(s.dim, integration.IndexFaissIvfFlat, metric.IP),
|
||||
})
|
||||
s.NoError(err)
|
||||
err = merr.Error(createIndexStatus)
|
||||
s.NoError(err)
|
||||
s.WaitForIndexBuilt(context.TODO(), collectionName, integration.FloatVecField)
|
||||
|
||||
// load
|
||||
loadStatus, err := c.Proxy.LoadCollection(context.TODO(), &milvuspb.LoadCollectionRequest{
|
||||
DbName: dbName,
|
||||
CollectionName: collectionName,
|
||||
})
|
||||
s.NoError(err)
|
||||
err = merr.Error(loadStatus)
|
||||
s.NoError(err)
|
||||
s.WaitForLoad(context.TODO(), collectionName)
|
||||
}
|
||||
|
||||
func (s *DataNodeCompatibility) checkCollections() bool {
|
||||
req := &milvuspb.ShowCollectionsRequest{
|
||||
DbName: "",
|
||||
TimeStamp: 0, // means now
|
||||
}
|
||||
resp, err := s.Cluster.Proxy.ShowCollections(context.TODO(), req)
|
||||
s.NoError(err)
|
||||
s.Equal(len(resp.CollectionIds), s.numCollections)
|
||||
notLoaded := 0
|
||||
loaded := 0
|
||||
for _, name := range resp.CollectionNames {
|
||||
loadProgress, err := s.Cluster.Proxy.GetLoadingProgress(context.TODO(), &milvuspb.GetLoadingProgressRequest{
|
||||
DbName: "",
|
||||
CollectionName: name,
|
||||
})
|
||||
s.NoError(err)
|
||||
if loadProgress.GetProgress() != int64(100) {
|
||||
notLoaded++
|
||||
} else {
|
||||
loaded++
|
||||
}
|
||||
}
|
||||
return notLoaded == 0
|
||||
}
|
||||
|
||||
func (s *DataNodeCompatibility) search(collectionName string, currentNumRows int) {
|
||||
c := s.Cluster
|
||||
var err error
|
||||
// Query
|
||||
queryReq := &milvuspb.QueryRequest{
|
||||
Base: nil,
|
||||
CollectionName: collectionName,
|
||||
PartitionNames: nil,
|
||||
Expr: "",
|
||||
OutputFields: []string{"count(*)"},
|
||||
TravelTimestamp: 0,
|
||||
GuaranteeTimestamp: 0,
|
||||
}
|
||||
queryResult, err := c.Proxy.Query(context.TODO(), queryReq)
|
||||
s.NoError(err)
|
||||
s.Equal(len(queryResult.FieldsData), 1)
|
||||
numEntities := queryResult.FieldsData[0].GetScalars().GetLongData().Data[0]
|
||||
s.Equal(numEntities, int64(currentNumRows))
|
||||
|
||||
// Search
|
||||
expr := fmt.Sprintf("%s > 0", integration.Int64Field)
|
||||
nq := 10
|
||||
topk := 10
|
||||
roundDecimal := -1
|
||||
radius := 10
|
||||
|
||||
params := integration.GetSearchParams(integration.IndexFaissIvfFlat, metric.IP)
|
||||
params["radius"] = radius
|
||||
searchReq := integration.ConstructSearchRequest("", collectionName, expr,
|
||||
integration.FloatVecField, schemapb.DataType_FloatVector, nil, metric.IP, params, nq, s.dim, topk, roundDecimal)
|
||||
|
||||
searchResult, _ := c.Proxy.Search(context.TODO(), searchReq)
|
||||
|
||||
err = merr.Error(searchResult.GetStatus())
|
||||
s.NoError(err)
|
||||
}
|
||||
|
||||
func (s *DataNodeCompatibility) insertBatchCollections(prefix string, collectionBatchSize, idxStart int, wg *sync.WaitGroup) {
|
||||
for idx := 0; idx < collectionBatchSize; idx++ {
|
||||
collectionName := prefix + "_" + strconv.Itoa(idxStart+idx)
|
||||
s.loadCollection(collectionName)
|
||||
}
|
||||
wg.Done()
|
||||
}
|
||||
|
||||
func (s *DataNodeCompatibility) insert(collectionName string, rowNum int) {
|
||||
fVecColumn := integration.NewFloatVectorFieldData(integration.FloatVecField, rowNum, s.dim)
|
||||
hashKeys := integration.GenerateHashKeys(rowNum)
|
||||
insertResult, err := s.Cluster.Proxy.Insert(context.TODO(), &milvuspb.InsertRequest{
|
||||
DbName: "",
|
||||
CollectionName: collectionName,
|
||||
FieldsData: []*schemapb.FieldData{fVecColumn},
|
||||
HashKeys: hashKeys,
|
||||
NumRows: uint32(rowNum),
|
||||
})
|
||||
s.NoError(err)
|
||||
s.True(merr.Ok(insertResult.GetStatus()))
|
||||
s.flush(collectionName)
|
||||
}
|
||||
|
||||
func (s *DataNodeCompatibility) insertAndCheck(collectionName string, currentNumRows *int, testInsert bool) {
|
||||
s.search(collectionName, *currentNumRows)
|
||||
insertRows := 1000
|
||||
if testInsert {
|
||||
s.insert(collectionName, insertRows)
|
||||
*currentNumRows += insertRows
|
||||
}
|
||||
s.search(collectionName, *currentNumRows)
|
||||
}
|
||||
|
||||
func (s *DataNodeCompatibility) setupData() {
|
||||
// Add the second data node
|
||||
s.Cluster.AddDataNode()
|
||||
goRoutineNum := s.maxGoRoutineNum
|
||||
if goRoutineNum > s.numCollections {
|
||||
goRoutineNum = s.numCollections
|
||||
}
|
||||
collectionBatchSize := s.numCollections / goRoutineNum
|
||||
log.Info(fmt.Sprintf("=========================test with dim=%d, s.rowsPerCollection=%d, s.numCollections=%d, goRoutineNum=%d==================", s.dim, s.rowsPerCollection, s.numCollections, goRoutineNum))
|
||||
log.Info("=========================Start to inject data=========================")
|
||||
s.prefix = "TestDataNodeUtil" + funcutil.GenRandomStr()
|
||||
searchName := s.prefix + "_0"
|
||||
wg := sync.WaitGroup{}
|
||||
for idx := 0; idx < goRoutineNum; idx++ {
|
||||
wg.Add(1)
|
||||
go s.insertBatchCollections(s.prefix, collectionBatchSize, idx*collectionBatchSize, &wg)
|
||||
}
|
||||
wg.Wait()
|
||||
log.Info("=========================Data injection finished=========================")
|
||||
s.checkCollections()
|
||||
log.Info(fmt.Sprintf("=========================start to search %s=========================", searchName))
|
||||
s.search(searchName, s.rowsPerCollection)
|
||||
log.Info("=========================Search finished=========================")
|
||||
time.Sleep(s.waitTimeInSec)
|
||||
s.checkCollections()
|
||||
log.Info(fmt.Sprintf("=========================start to search2 %s=========================", searchName))
|
||||
s.search(searchName, s.rowsPerCollection)
|
||||
log.Info("=========================Search2 finished=========================")
|
||||
s.checkAllCollectionsReady()
|
||||
}
|
||||
|
||||
func (s *DataNodeCompatibility) checkAllCollectionsReady() {
|
||||
goRoutineNum := s.maxGoRoutineNum
|
||||
if goRoutineNum > s.numCollections {
|
||||
goRoutineNum = s.numCollections
|
||||
}
|
||||
collectionBatchSize := s.numCollections / goRoutineNum
|
||||
for i := 0; i < goRoutineNum; i++ {
|
||||
for idx := 0; idx < collectionBatchSize; idx++ {
|
||||
collectionName := s.prefix + "_" + strconv.Itoa(i*collectionBatchSize+idx)
|
||||
s.search(collectionName, s.rowsPerCollection)
|
||||
queryReq := &milvuspb.QueryRequest{
|
||||
CollectionName: collectionName,
|
||||
Expr: "",
|
||||
OutputFields: []string{"count(*)"},
|
||||
}
|
||||
_, err := s.Cluster.Proxy.Query(context.TODO(), queryReq)
|
||||
s.NoError(err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (s *DataNodeCompatibility) checkSingleDNRestarts(currentNumRows *int, numNodes, idx int, testInsert bool) {
|
||||
// Stop all data nodes
|
||||
s.Cluster.StopAllDataNodes()
|
||||
// Add new data nodes.
|
||||
var dn []*grpcdatanode.Server
|
||||
for i := 0; i < numNodes; i++ {
|
||||
dn = append(dn, s.Cluster.AddDataNode())
|
||||
}
|
||||
time.Sleep(s.waitTimeInSec)
|
||||
cn := fmt.Sprintf("%s_0", s.prefix)
|
||||
s.insertAndCheck(cn, currentNumRows, testInsert)
|
||||
dn[idx].Stop()
|
||||
time.Sleep(s.waitTimeInSec)
|
||||
s.insertAndCheck(cn, currentNumRows, testInsert)
|
||||
}
|
||||
|
||||
func (s *DataNodeCompatibility) checkDNRestarts(currentNumRows *int, testInsert bool) {
|
||||
numDatanodes := 2 // configurable
|
||||
for idx := 0; idx < numDatanodes; idx++ {
|
||||
s.checkSingleDNRestarts(currentNumRows, numDatanodes, idx, testInsert)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *DataNodeCompatibility) restartDC() {
|
||||
c := s.Cluster
|
||||
c.DataCoord.Stop()
|
||||
c.DataCoord = grpcdatacoord.NewServer(context.TODO(), c.GetFactory())
|
||||
err := c.DataCoord.Run()
|
||||
s.NoError(err)
|
||||
}
|
||||
|
||||
func (s *DataNodeCompatibility) TestCompatibility() {
|
||||
s.setupParam()
|
||||
s.setupData()
|
||||
rows := s.rowsPerCollection
|
||||
|
||||
// new coord + new node
|
||||
s.checkDNRestarts(&rows, true)
|
||||
|
||||
// new coord + old node
|
||||
paramtable.Get().Save(paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.Key, "false")
|
||||
s.checkDNRestarts(&rows, false)
|
||||
|
||||
// old coord + old node
|
||||
s.restartDC()
|
||||
s.checkDNRestarts(&rows, true)
|
||||
|
||||
// old coord + new node
|
||||
paramtable.Get().Save(paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.Key, "true")
|
||||
s.checkDNRestarts(&rows, false)
|
||||
|
||||
// new coord + both old & new datanodes.
|
||||
paramtable.Get().Save(paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.Key, "false")
|
||||
s.restartDC()
|
||||
s.Cluster.StopAllDataNodes()
|
||||
d1 := s.Cluster.AddDataNode()
|
||||
d2 := s.Cluster.AddDataNode()
|
||||
cn := fmt.Sprintf("%s_0", s.prefix)
|
||||
s.insertAndCheck(cn, &rows, true)
|
||||
paramtable.Get().Save(paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.Key, "true")
|
||||
s.restartDC()
|
||||
s.insertAndCheck(cn, &rows, false)
|
||||
s.Cluster.AddDataNode()
|
||||
d1.Stop()
|
||||
s.checkDNRestarts(&rows, true)
|
||||
s.Cluster.AddDataNode()
|
||||
d2.Stop()
|
||||
s.checkDNRestarts(&rows, true)
|
||||
}
|
||||
|
||||
func TestDataNodeCompatibility(t *testing.T) {
|
||||
suite.Run(t, new(DataNodeCompatibility))
|
||||
}
|
Loading…
Reference in New Issue
Block a user