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:
jaime 2024-07-01 14:46:07 +08:00 committed by GitHub
parent 14e827dc6c
commit d6afb31b94
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
57 changed files with 959 additions and 5767 deletions

View File

@ -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=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=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=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=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 --output=$(PWD)/internal/datanode --filename=mock_channelmanager.go --with-expecter --structname=MockChannelManager --outpkg=datanode --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 $(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 generate-mockery-metastore: getdeps

View File

@ -419,7 +419,6 @@ indexNode:
dataCoord: dataCoord:
channel: channel:
watchTimeoutInterval: 300 # Timeout on watching channels (in seconds). Datanode tickler update watch progress will reset timeout timer. 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 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 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 balanceInterval: 360 # The interval with which the channel manager check dml channel balance status

View File

@ -26,7 +26,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/paramtable"
) )
type ROChannel interface { type ROChannel interface {
@ -50,17 +49,7 @@ func NewRWChannel(name string,
schema *schemapb.CollectionSchema, schema *schemapb.CollectionSchema,
createTs uint64, createTs uint64,
) RWChannel { ) RWChannel {
if paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.GetAsBool() { return &StateChannel{
return &StateChannel{
Name: name,
CollectionID: collectionID,
StartPositions: startPos,
Schema: schema,
CreateTimestamp: createTs,
}
}
return &channelMeta{
Name: name, Name: name,
CollectionID: collectionID, CollectionID: collectionID,
StartPositions: startPos, StartPositions: startPos,

View File

@ -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
}
}

View File

@ -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))
}
})
}

View File

@ -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

View File

@ -32,7 +32,6 @@ import (
"github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/kv"
"github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics" "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/timerecord"
"github.com/milvus-io/milvus/pkg/util/typeutil" "github.com/milvus-io/milvus/pkg/util/typeutil"
) )
@ -154,9 +153,7 @@ func (op *ChannelOp) BuildKV() (map[string]string, []string, error) {
switch op.Type { switch op.Type {
case Add, Watch, Release: case Add, Watch, Release:
tmpWatchInfo := proto.Clone(ch.GetWatchInfo()).(*datapb.ChannelWatchInfo) 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) info, err := proto.Marshal(tmpWatchInfo)
if err != nil { if err != nil {
return saves, removals, err return saves, removals, err

View File

@ -22,91 +22,15 @@ import (
"testing" "testing"
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock" "github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite" "github.com/stretchr/testify/suite"
"github.com/milvus-io/milvus/internal/kv/mocks" "github.com/milvus-io/milvus/internal/kv/mocks"
"github.com/milvus-io/milvus/internal/proto/datapb" "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/metrics"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/testutils" "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 { type ChannelStoreReloadSuite struct {
testutils.PromMetricsSuite testutils.PromMetricsSuite

View File

@ -487,16 +487,9 @@ func (s *Server) initCluster() error {
s.sessionManager = NewSessionManagerImpl(withSessionCreator(s.dataNodeCreator)) s.sessionManager = NewSessionManagerImpl(withSessionCreator(s.dataNodeCreator))
var err error var err error
if paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.GetAsBool() { s.channelManager, err = NewChannelManagerV2(s.watchClient, s.handler, s.sessionManager, s.allocator, withCheckerV2())
s.channelManager, err = NewChannelManagerV2(s.watchClient, s.handler, s.sessionManager, s.allocator, withCheckerV2()) if err != nil {
if err != nil { return err
return err
}
} else {
s.channelManager, err = NewChannelManager(s.watchClient, s.handler, withMsgstreamFactory(s.factory), withStateChecker(), withBgChecker())
if err != nil {
return err
}
} }
s.cluster = NewClusterImpl(s.sessionManager, s.channelManager) s.cluster = NewClusterImpl(s.sessionManager, s.channelManager)
return nil return nil

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package channel
import ( import (
"context" "context"
@ -22,19 +22,21 @@ import (
"time" "time"
"github.com/cockroachdb/errors" "github.com/cockroachdb/errors"
"go.uber.org/atomic"
"go.uber.org/zap" "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/internal/proto/datapb"
"github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/lifetime" "github.com/milvus-io/milvus/pkg/util/lifetime"
"github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/typeutil" "github.com/milvus-io/milvus/pkg/util/typeutil"
) )
type ( type (
releaseFunc func(channel string) 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 { type ChannelManager interface {
@ -45,10 +47,10 @@ type ChannelManager interface {
} }
type ChannelManagerImpl struct { type ChannelManagerImpl struct {
mu sync.RWMutex mu sync.RWMutex
dn *DataNode pipelineParams *util.PipelineParams
fgManager FlowgraphManager fgManager pipeline.FlowgraphManager
communicateCh chan *opState communicateCh chan *opState
opRunners *typeutil.ConcurrentMap[string, *opRunner] // channel -> runner opRunners *typeutil.ConcurrentMap[string, *opRunner] // channel -> runner
@ -60,16 +62,16 @@ type ChannelManagerImpl struct {
closeWaiter sync.WaitGroup closeWaiter sync.WaitGroup
} }
func NewChannelManager(dn *DataNode) *ChannelManagerImpl { func NewChannelManager(pipelineParams *util.PipelineParams, fgManager pipeline.FlowgraphManager) *ChannelManagerImpl {
cm := ChannelManagerImpl{ cm := ChannelManagerImpl{
dn: dn, pipelineParams: pipelineParams,
fgManager: dn.flowgraphManager, fgManager: fgManager,
communicateCh: make(chan *opState, 100), communicateCh: make(chan *opState, 100),
opRunners: typeutil.NewConcurrentMap[string, *opRunner](), opRunners: typeutil.NewConcurrentMap[string, *opRunner](),
abnormals: typeutil.NewConcurrentMap[int64, string](), abnormals: typeutil.NewConcurrentMap[int64, string](),
releaseFunc: dn.flowgraphManager.RemoveFlowgraph, releaseFunc: fgManager.RemoveFlowgraph,
closeCh: lifetime.NewSafeChan(), closeCh: lifetime.NewSafeChan(),
} }
@ -209,7 +211,7 @@ func (m *ChannelManagerImpl) handleOpState(opState *opState) {
} }
func (m *ChannelManagerImpl) getOrCreateRunner(channel string) *opRunner { 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 { if !loaded {
runner.Start() runner.Start()
} }
@ -224,17 +226,17 @@ func (m *ChannelManagerImpl) finishOp(opID int64, channel string) {
} }
type opInfo struct { type opInfo struct {
tickler *tickler tickler *util.Tickler
} }
type opRunner struct { type opRunner struct {
channel string channel string
dn *DataNode pipelineParams *util.PipelineParams
releaseFunc releaseFunc releaseFunc releaseFunc
watchFunc watchFunc watchFunc watchFunc
guard sync.RWMutex guard sync.RWMutex
allOps map[UniqueID]*opInfo // opID -> tickler allOps map[util.UniqueID]*opInfo // opID -> tickler
opsInQueue chan *datapb.ChannelWatchInfo opsInQueue chan *datapb.ChannelWatchInfo
resultCh chan *opState resultCh chan *opState
@ -242,16 +244,16 @@ type opRunner struct {
closeWg sync.WaitGroup 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{ return &opRunner{
channel: channel, channel: channel,
dn: dn, pipelineParams: pipelineParams,
releaseFunc: releaseF, releaseFunc: releaseF,
watchFunc: watchF, watchFunc: watchF,
opsInQueue: make(chan *datapb.ChannelWatchInfo, 10), opsInQueue: make(chan *datapb.ChannelWatchInfo, 10),
allOps: make(map[UniqueID]*opInfo), allOps: make(map[util.UniqueID]*opInfo),
resultCh: resultCh, resultCh: resultCh,
closeCh: lifetime.NewSafeChan(), 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() r.guard.Lock()
defer r.guard.Unlock() defer r.guard.Unlock()
delete(r.allOps, opID) delete(r.allOps, opID)
} }
func (r *opRunner) Exist(opID UniqueID) bool { func (r *opRunner) Exist(opID util.UniqueID) bool {
r.guard.RLock() r.guard.RLock()
defer r.guard.RUnlock() defer r.guard.RUnlock()
_, ok := r.allOps[opID] _, ok := r.allOps[opID]
@ -334,7 +336,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState {
opState.state = datapb.ChannelWatchState_WatchFailure opState.state = datapb.ChannelWatchState_WatchFailure
return opState return opState
} }
tickler := newTickler() tickler := util.NewTickler()
opInfo.tickler = tickler opInfo.tickler = tickler
var ( var (
@ -342,7 +344,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState {
finishWaiter sync.WaitGroup 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) ctx, cancel := context.WithTimeout(context.Background(), watchTimeout)
defer cancel() defer cancel()
@ -358,19 +360,19 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState {
select { select {
case <-timer.C: case <-timer.C:
// watch timeout // watch timeout
tickler.close() tickler.Close()
cancel() cancel()
log.Info("Stop timer for ToWatch operation timeout") log.Info("Stop timer for ToWatch operation timeout")
return return
case <-r.closeCh.CloseCh(): case <-r.closeCh.CloseCh():
// runner closed from outside // runner closed from outside
tickler.close() tickler.Close()
cancel() cancel()
log.Info("Suspend ToWatch operation from outside of opRunner") log.Info("Suspend ToWatch operation from outside of opRunner")
return return
case <-tickler.progressSig: case <-tickler.GetProgressSig():
log.Info("Reset timer for tickler updated") log.Info("Reset timer for tickler updated")
timer.Reset(watchTimeout) timer.Reset(watchTimeout)
@ -387,7 +389,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState {
go func() { go func() {
defer finishWaiter.Done() defer finishWaiter.Done()
fg, err := r.watchFunc(ctx, r.dn, info, tickler) fg, err := r.watchFunc(ctx, r.pipelineParams, info, tickler)
if err != nil { if err != nil {
opState.state = datapb.ChannelWatchState_WatchFailure opState.state = datapb.ChannelWatchState_WatchFailure
} else { } else {
@ -402,7 +404,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState {
} }
// releaseWithTimer will return ReleaseFailure after WatchTimeoutInterval // 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{ opState := &opState{
channel: channel, channel: channel,
opID: opID, opID: opID,
@ -416,7 +418,7 @@ func (r *opRunner) releaseWithTimer(releaseFunc releaseFunc, channel string, opI
startTimer := func(finishWaiter *sync.WaitGroup) { startTimer := func(finishWaiter *sync.WaitGroup) {
defer finishWaiter.Done() defer finishWaiter.Done()
releaseTimeout := Params.DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second) releaseTimeout := paramtable.Get().DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second)
timer := time.NewTimer(releaseTimeout) timer := time.NewTimer(releaseTimeout)
defer timer.Stop() defer timer.Stop()
@ -474,61 +476,17 @@ type opState struct {
channel string channel string
opID int64 opID int64
state datapb.ChannelWatchState state datapb.ChannelWatchState
fg *dataSyncService fg *pipeline.DataSyncService
} }
// executeWatch will always return, won't be stuck, either success or fail. // 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) { func executeWatch(ctx context.Context, pipelineParams *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler) (*pipeline.DataSyncService, error) {
dataSyncService, err := newDataSyncService(ctx, dn, info, tickler) dataSyncService, err := pipeline.NewDataSyncService(ctx, pipelineParams, info, tickler)
if err != nil { if err != nil {
return nil, err return nil, err
} }
dataSyncService.start() dataSyncService.Start()
return dataSyncService, nil 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),
}
}

View File

@ -14,24 +14,43 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package channel
import ( import (
"context" "context"
"os"
"testing" "testing"
"github.com/cockroachdb/errors" "github.com/cockroachdb/errors"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite" "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/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/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/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/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) { func TestChannelManagerSuite(t *testing.T) {
suite.Run(t, new(ChannelManagerSuite)) suite.Run(t, new(ChannelManagerSuite))
} }
@ -41,11 +60,30 @@ func TestOpRunnerSuite(t *testing.T) {
} }
func (s *OpRunnerSuite) SetupTest() { func (s *OpRunnerSuite) SetupTest() {
ctx := context.Background() mockedBroker := broker.NewMockBroker(s.T())
s.mockAlloc = allocator.NewMockAllocator(s.T()) mockedBroker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).
Return([]*datapb.SegmentInfo{}, nil).Maybe()
s.node = newIDLEDataNodeMock(ctx, schemapb.DataType_Int64) wbManager := writebuffer.NewMockBufferManager(s.T())
s.node.allocator = s.mockAlloc 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() { func (s *OpRunnerSuite) TestWatchWithTimer() {
@ -53,11 +91,12 @@ func (s *OpRunnerSuite) TestWatchWithTimer() {
channel string = "ch-1" channel string = "ch-1"
commuCh = make(chan *opState) commuCh = make(chan *opState)
) )
info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
mockReleaseFunc := func(channel string) { mockReleaseFunc := func(channel string) {
log.Info("mock release func") 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) err := runner.Enqueue(info)
s.Require().NoError(err) s.Require().NoError(err)
@ -70,21 +109,21 @@ func (s *OpRunnerSuite) TestWatchWithTimer() {
func (s *OpRunnerSuite) TestWatchTimeout() { func (s *OpRunnerSuite) TestWatchTimeout() {
channel := "by-dev-rootcoord-dml-1000" channel := "by-dev-rootcoord-dml-1000"
paramtable.Get().Save(Params.DataCoordCfg.WatchTimeoutInterval.Key, "0.000001") paramtable.Get().Save(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key, "0.000001")
defer paramtable.Get().Reset(Params.DataCoordCfg.WatchTimeoutInterval.Key) defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key)
info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
sig := make(chan struct{}) sig := make(chan struct{})
commuCh := make(chan *opState) commuCh := make(chan *opState)
mockReleaseFunc := func(channel string) { log.Info("mock release func") } 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() <-ctx.Done()
sig <- struct{}{} sig <- struct{}{}
return nil, errors.New("timeout") return nil, errors.New("timeout")
} }
runner := NewOpRunner(channel, s.node, mockReleaseFunc, mockWatchFunc, commuCh) runner := NewOpRunner(channel, s.pipelineParams, mockReleaseFunc, mockWatchFunc, commuCh)
runner.Start() runner.Start()
defer runner.Close() defer runner.Close()
err := runner.Enqueue(info) err := runner.Enqueue(info)
@ -99,55 +138,40 @@ func (s *OpRunnerSuite) TestWatchTimeout() {
type OpRunnerSuite struct { type OpRunnerSuite struct {
suite.Suite suite.Suite
node *DataNode pipelineParams *util.PipelineParams
mockAlloc *allocator.MockAllocator
} }
type ChannelManagerSuite struct { type ChannelManagerSuite struct {
suite.Suite suite.Suite
node *DataNode pipelineParams *util.PipelineParams
manager *ChannelManagerImpl manager *ChannelManagerImpl
} }
func (s *ChannelManagerSuite) SetupTest() { func (s *ChannelManagerSuite) SetupTest() {
ctx := context.Background() factory := dependency.NewDefaultFactory(true)
s.node = newIDLEDataNodeMock(ctx, schemapb.DataType_Int64)
s.node.allocator = allocator.NewMockAllocator(s.T())
s.node.flowgraphManager = newFlowgraphManager()
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 { mockedBroker := &broker.MockBroker{}
return &datapb.ChannelWatchInfo{ mockedBroker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).Return([]*datapb.SegmentInfo{}, nil).Maybe()
OpID: opID,
State: state, s.pipelineParams = &util.PipelineParams{
Vchan: &datapb.VchannelInfo{ Ctx: context.TODO(),
CollectionID: 1, Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 0}},
ChannelName: channel, WriteBufferManager: wbManager,
}, Broker: mockedBroker,
Schema: &schemapb.CollectionSchema{ MsgStreamFactory: factory,
Name: "test_collection", DispClient: msgdispatcher.NewClient(factory, typeutil.DataNodeRole, paramtable.GetNodeID()),
Fields: []*schemapb.FieldSchema{ SyncMgr: syncmgr.NewMockSyncManager(s.T()),
{ Allocator: allocator.NewMockAllocator(s.T()),
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"},
},
},
},
},
} }
s.manager = NewChannelManager(s.pipelineParams, pipeline.NewFlowgraphManager())
} }
func (s *ChannelManagerSuite) TearDownTest() { func (s *ChannelManagerSuite) TearDownTest() {
@ -165,7 +189,7 @@ func (s *ChannelManagerSuite) TestReleaseStuck() {
stuckSig <- struct{}{} 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()) s.Require().Equal(0, s.manager.opRunners.Len())
err := s.manager.Submit(info) err := s.manager.Submit(info)
s.Require().NoError(err) s.Require().NoError(err)
@ -175,9 +199,9 @@ func (s *ChannelManagerSuite) TestReleaseStuck() {
s.manager.handleOpState(opState) s.manager.handleOpState(opState)
releaseInfo := getWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease) releaseInfo := util.GetWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease)
paramtable.Get().Save(Params.DataCoordCfg.WatchTimeoutInterval.Key, "0.1") paramtable.Get().Save(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key, "0.1")
defer paramtable.Get().Reset(Params.DataCoordCfg.WatchTimeoutInterval.Key) defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key)
err = s.manager.Submit(releaseInfo) err = s.manager.Submit(releaseInfo)
s.NoError(err) s.NoError(err)
@ -201,7 +225,7 @@ func (s *ChannelManagerSuite) TestReleaseStuck() {
func (s *ChannelManagerSuite) TestSubmitIdempotent() { func (s *ChannelManagerSuite) TestSubmitIdempotent() {
channel := "by-dev-rootcoord-dml-1" 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()) s.Require().Equal(0, s.manager.opRunners.Len())
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
@ -220,7 +244,7 @@ func (s *ChannelManagerSuite) TestSubmitIdempotent() {
func (s *ChannelManagerSuite) TestSubmitSkip() { func (s *ChannelManagerSuite) TestSubmitSkip() {
channel := "by-dev-rootcoord-dml-1" 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()) s.Require().Equal(0, s.manager.opRunners.Len())
err := s.manager.Submit(info) err := s.manager.Submit(info)
@ -232,7 +256,7 @@ func (s *ChannelManagerSuite) TestSubmitSkip() {
s.NotNil(opState) s.NotNil(opState)
s.Equal(datapb.ChannelWatchState_WatchSuccess, opState.state) s.Equal(datapb.ChannelWatchState_WatchSuccess, opState.state)
s.NotNil(opState.fg) s.NotNil(opState.fg)
s.Equal(info.GetOpID(), opState.fg.opID) s.Equal(info.GetOpID(), opState.fg.GetOpID())
s.manager.handleOpState(opState) s.manager.handleOpState(opState)
err = s.manager.Submit(info) err = s.manager.Submit(info)
@ -247,7 +271,7 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() {
channel := "by-dev-rootcoord-dml-0" channel := "by-dev-rootcoord-dml-0"
// watch // watch
info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
err := s.manager.Submit(info) err := s.manager.Submit(info)
s.NoError(err) s.NoError(err)
@ -256,7 +280,7 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() {
s.NotNil(opState) s.NotNil(opState)
s.Equal(datapb.ChannelWatchState_WatchSuccess, opState.state) s.Equal(datapb.ChannelWatchState_WatchSuccess, opState.state)
s.NotNil(opState.fg) s.NotNil(opState.fg)
s.Equal(info.GetOpID(), opState.fg.opID) s.Equal(info.GetOpID(), opState.fg.GetOpID())
resp := s.manager.GetProgress(info) resp := s.manager.GetProgress(info)
s.Equal(info.GetOpID(), resp.GetOpID()) s.Equal(info.GetOpID(), resp.GetOpID())
@ -272,7 +296,7 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() {
s.Equal(datapb.ChannelWatchState_WatchSuccess, resp.GetState()) s.Equal(datapb.ChannelWatchState_WatchSuccess, resp.GetState())
// release // release
info = getWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease) info = util.GetWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease)
err = s.manager.Submit(info) err = s.manager.Submit(info)
s.NoError(err) s.NoError(err)

View File

@ -1,6 +1,6 @@
// Code generated by mockery v2.32.4. DO NOT EDIT. // Code generated by mockery v2.32.4. DO NOT EDIT.
package datanode package channel
import ( import (
datapb "github.com/milvus-io/milvus/internal/proto/datapb" datapb "github.com/milvus-io/milvus/internal/proto/datapb"

View File

@ -30,7 +30,6 @@ import (
"github.com/milvus-io/milvus/internal/datanode/allocator" "github.com/milvus-io/milvus/internal/datanode/allocator"
"github.com/milvus-io/milvus/internal/datanode/io" "github.com/milvus-io/milvus/internal/datanode/io"
"github.com/milvus-io/milvus/internal/datanode/metacache" "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/metastore/kv/binlog"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/storage" "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) { future := pool.Submit(func() (any, error) {
_ = binlog.DecompressBinLog(storage.StatsBinlog, segment.GetCollectionID(), _ = binlog.DecompressBinLog(storage.StatsBinlog, segment.GetCollectionID(),
segment.GetPartitionID(), segment.GetSegmentID(), segment.GetField2StatslogPaths()) 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()) t.plan.GetSchema(), segment.GetSegmentID(), segment.GetField2StatslogPaths())
if err != nil { if err != nil {
log.Warn("failed to load segment stats log", log.Warn("failed to load segment stats log",

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package util package compaction
import ( import (
"context" "context"

View File

@ -36,9 +36,12 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "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/allocator"
"github.com/milvus-io/milvus/internal/datanode/broker" "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/compaction"
"github.com/milvus-io/milvus/internal/datanode/importv2" "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/syncmgr"
"github.com/milvus-io/milvus/internal/datanode/util"
"github.com/milvus-io/milvus/internal/datanode/writebuffer" "github.com/milvus-io/milvus/internal/datanode/writebuffer"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/storage" "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/conc"
"github.com/milvus-io/milvus/pkg/util/expr" "github.com/milvus-io/milvus/pkg/util/expr"
"github.com/milvus-io/milvus/pkg/util/logutil" "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/paramtable"
"github.com/milvus-io/milvus/pkg/util/retry" "github.com/milvus-io/milvus/pkg/util/retry"
"github.com/milvus-io/milvus/pkg/util/typeutil" "github.com/milvus-io/milvus/pkg/util/typeutil"
@ -63,8 +65,6 @@ const (
ConnectEtcdMaxRetryTime = 100 ConnectEtcdMaxRetryTime = 100
) )
var getFlowGraphServiceAttempts = uint(50)
// makes sure DataNode implements types.DataNode // makes sure DataNode implements types.DataNode
var _ types.DataNode = (*DataNode)(nil) var _ types.DataNode = (*DataNode)(nil)
@ -88,21 +88,19 @@ type DataNode struct {
cancel context.CancelFunc cancel context.CancelFunc
Role string Role string
stateCode atomic.Value // commonpb.StateCode_Initializing stateCode atomic.Value // commonpb.StateCode_Initializing
flowgraphManager FlowgraphManager flowgraphManager pipeline.FlowgraphManager
eventManager *EventManager channelManager channel.ChannelManager
channelManager ChannelManager
syncMgr syncmgr.SyncManager syncMgr syncmgr.SyncManager
writeBufferManager writebuffer.BufferManager writeBufferManager writebuffer.BufferManager
importTaskMgr importv2.TaskManager importTaskMgr importv2.TaskManager
importScheduler importv2.Scheduler importScheduler importv2.Scheduler
clearSignal chan string // vchannel name segmentCache *util.Cache
segmentCache *Cache
compactionExecutor compaction.Executor compactionExecutor compaction.Executor
timeTickSender *timeTickSender timeTickSender *util.TimeTickSender
channelCheckpointUpdater *channelCheckpointUpdater channelCheckpointUpdater *util.ChannelCheckpointUpdater
etcdCli *clientv3.Client etcdCli *clientv3.Client
address string address string
@ -114,7 +112,6 @@ type DataNode struct {
initOnce sync.Once initOnce sync.Once
startOnce sync.Once startOnce sync.Once
stopOnce sync.Once stopOnce sync.Once
stopWaiter sync.WaitGroup
sessionMu sync.Mutex // to fix data race sessionMu sync.Mutex // to fix data race
session *sessionutil.Session session *sessionutil.Session
watchKv kv.WatchKV watchKv kv.WatchKV
@ -139,14 +136,11 @@ func NewDataNode(ctx context.Context, factory dependency.Factory) *DataNode {
cancel: cancel2, cancel: cancel2,
Role: typeutil.DataNodeRole, Role: typeutil.DataNodeRole,
rootCoord: nil, rootCoord: nil,
dataCoord: nil, dataCoord: nil,
factory: factory, factory: factory,
segmentCache: newCache(), segmentCache: util.NewCache(),
compactionExecutor: compaction.NewExecutor(), compactionExecutor: compaction.NewExecutor(),
clearSignal: make(chan string, 100),
reportImportRetryTimes: 10, reportImportRetryTimes: 10,
} }
node.UpdateStateCode(commonpb.StateCode_Abnormal) node.UpdateStateCode(commonpb.StateCode_Abnormal)
@ -215,17 +209,6 @@ func (node *DataNode) initSession() error {
return nil 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 { func (node *DataNode) GetNodeID() int64 {
if node.session != nil { if node.session != nil {
return node.session.ServerID return node.session.ServerID
@ -250,7 +233,7 @@ func (node *DataNode) Init() error {
node.broker = broker.NewCoordBroker(node.dataCoord, serverID) node.broker = broker.NewCoordBroker(node.dataCoord, serverID)
err := node.initRateCollector() err := util.InitGlobalRateCollector()
if err != nil { if err != nil {
log.Error("DataNode server init rateCollector failed", zap.Error(err)) log.Error("DataNode server init rateCollector failed", zap.Error(err))
initError = err initError = err
@ -292,38 +275,14 @@ func (node *DataNode) Init() error {
node.importTaskMgr = importv2.NewTaskManager() node.importTaskMgr = importv2.NewTaskManager()
node.importScheduler = importv2.NewScheduler(node.importTaskMgr) node.importScheduler = importv2.NewScheduler(node.importTaskMgr)
node.channelCheckpointUpdater = newChannelCheckpointUpdater(node.broker) node.channelCheckpointUpdater = util.NewChannelCheckpointUpdater(node.broker)
node.flowgraphManager = newFlowgraphManager() 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)) log.Info("init datanode done", zap.String("Address", node.address))
}) })
return initError 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 // tryToReleaseFlowgraph tries to release a flowgraph
func (node *DataNode) tryToReleaseFlowgraph(channel string) { func (node *DataNode) tryToReleaseFlowgraph(channel string) {
log.Info("try to release flowgraph", zap.String("channel", channel)) 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 // Start will update DataNode state to HEALTHY
func (node *DataNode) Start() error { func (node *DataNode) Start() error {
var startErr error var startErr error
@ -365,21 +308,6 @@ func (node *DataNode) Start() error {
} }
log.Info("start id allocator done", zap.String("role", typeutil.DataNodeRole)) 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 { connectEtcdFn := func() error {
etcdKV := etcdkv.NewEtcdKV(node.etcdCli, Params.EtcdCfg.MetaRootPath.GetValue(), etcdKV := etcdkv.NewEtcdKV(node.etcdCli, Params.EtcdCfg.MetaRootPath.GetValue(),
etcdkv.WithRequestTimeout(paramtable.Get().ServiceParam.EtcdCfg.RequestTimeout.GetAsDuration(time.Millisecond))) etcdkv.WithRequestTimeout(paramtable.Get().ServiceParam.EtcdCfg.RequestTimeout.GetAsDuration(time.Millisecond)))
@ -394,27 +322,20 @@ func (node *DataNode) Start() error {
node.writeBufferManager.Start() node.writeBufferManager.Start()
node.stopWaiter.Add(1)
go node.BackGroundGC(node.clearSignal)
go node.compactionExecutor.Start(node.ctx) go node.compactionExecutor.Start(node.ctx)
go node.importScheduler.Start() go node.importScheduler.Start()
if Params.DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() { 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)) 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 = channel.NewChannelManager(getPipelineParams(node), node.flowgraphManager)
node.channelManager.Start() node.channelManager.Start()
} else {
// Start node watch node
node.startWatchChannelsAtBackground(node.ctx)
}
node.UpdateStateCode(commonpb.StateCode_Healthy) node.UpdateStateCode(commonpb.StateCode_Healthy)
}) })
@ -452,10 +373,6 @@ func (node *DataNode) Stop() error {
node.channelManager.Close() node.channelManager.Close()
} }
if node.eventManager != nil {
node.eventManager.CloseAll()
}
if node.writeBufferManager != nil { if node.writeBufferManager != nil {
node.writeBufferManager.Stop() node.writeBufferManager.Stop()
} }
@ -478,7 +395,7 @@ func (node *DataNode) Stop() error {
} }
if node.channelCheckpointUpdater != nil { if node.channelCheckpointUpdater != nil {
node.channelCheckpointUpdater.close() node.channelCheckpointUpdater.Close()
} }
if node.importScheduler != nil { 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 // Delay the cancellation of ctx to ensure that the session is automatically recycled after closed the flow graph
node.cancel() node.cancel()
node.stopWaiter.Wait()
}) })
return nil return nil
} }
// to fix data race // SetSession to fix data race
func (node *DataNode) SetSession(session *sessionutil.Session) { func (node *DataNode) SetSession(session *sessionutil.Session) {
node.sessionMu.Lock() node.sessionMu.Lock()
defer node.sessionMu.Unlock() defer node.sessionMu.Unlock()
node.session = session node.session = session
} }
// to fix data race // GetSession to fix data race
func (node *DataNode) GetSession() *sessionutil.Session { func (node *DataNode) GetSession() *sessionutil.Session {
node.sessionMu.Lock() node.sessionMu.Lock()
defer node.sessionMu.Unlock() defer node.sessionMu.Unlock()
return node.session 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,
}
}

View File

@ -32,21 +32,23 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "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-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/datanode/broker" "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/proto/datapb"
"github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types" "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/internal/util/sessionutil"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log" "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/etcd"
"github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable" "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) { func TestMain(t *testing.M) {
rand.Seed(time.Now().Unix()) rand.Seed(time.Now().Unix())
// init embed etcd // 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.EtcdCfg.Endpoints.Key, strings.Join(addrs, ","))
paramtable.Get().Save(Params.CommonCfg.DataCoordTimeTick.Key, Params.CommonCfg.DataCoordTimeTick.GetValue()+strconv.Itoa(rand.Int())) paramtable.Get().Save(Params.CommonCfg.DataCoordTimeTick.Key, Params.CommonCfg.DataCoordTimeTick.GetValue()+strconv.Itoa(rand.Int()))
rateCol, err = newRateCollector() err = util.InitGlobalRateCollector()
if err != nil { if err != nil {
panic("init test failed, err = " + err.Error()) panic("init test failed, err = " + err.Error())
} }
@ -79,11 +81,31 @@ func TestMain(t *testing.M) {
os.Exit(code) 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) { func TestDataNode(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
defer cancel() defer cancel()
node := NewIDLEDataNodeMock(ctx, schemapb.DataType_Int64)
node := newIDLEDataNodeMock(ctx, schemapb.DataType_Int64)
etcdCli, err := etcd.GetEtcdClient( etcdCli, err := etcd.GetEtcdClient(
Params.EtcdCfg.UseEmbedEtcd.GetAsBool(), Params.EtcdCfg.UseEmbedEtcd.GetAsBool(),
Params.EtcdCfg.EtcdUseSSL.GetAsBool(), Params.EtcdCfg.EtcdUseSSL.GetAsBool(),
@ -123,7 +145,7 @@ func TestDataNode(t *testing.T) {
description string description string
}{ }{
{nil, false, "nil input"}, {nil, false, "nil input"},
{&RootCoordFactory{}, true, "valid input"}, {&util.RootCoordFactory{}, true, "valid input"},
} }
for _, test := range tests { for _, test := range tests {
@ -146,7 +168,7 @@ func TestDataNode(t *testing.T) {
description string description string
}{ }{
{nil, false, "nil input"}, {nil, false, "nil input"},
{&DataCoordFactory{}, true, "valid input"}, {&util.DataCoordFactory{}, true, "valid input"},
} }
for _, test := range tests { for _, test := range tests {
@ -164,7 +186,7 @@ func TestDataNode(t *testing.T) {
t.Run("Test getSystemInfoMetrics", func(t *testing.T) { t.Run("Test getSystemInfoMetrics", func(t *testing.T) {
emptyNode := &DataNode{} emptyNode := &DataNode{}
emptyNode.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}}) emptyNode.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}})
emptyNode.flowgraphManager = newFlowgraphManager() emptyNode.flowgraphManager = pipeline.NewFlowgraphManager()
req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics) req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics)
assert.NoError(t, err) assert.NoError(t, err)
@ -179,64 +201,14 @@ func TestDataNode(t *testing.T) {
t.Run("Test getSystemInfoMetrics with quotaMetric error", func(t *testing.T) { t.Run("Test getSystemInfoMetrics with quotaMetric error", func(t *testing.T) {
emptyNode := &DataNode{} emptyNode := &DataNode{}
emptyNode.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}}) emptyNode.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}})
emptyNode.flowgraphManager = newFlowgraphManager() emptyNode.flowgraphManager = pipeline.NewFlowgraphManager()
req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics) req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics)
assert.NoError(t, err) assert.NoError(t, err)
rateCol.Deregister(metricsinfo.InsertConsumeThroughput) util.DeregisterRateCollector(metricsinfo.InsertConsumeThroughput)
resp, err := emptyNode.getSystemInfoMetrics(context.TODO(), req) resp, err := emptyNode.getSystemInfoMetrics(context.TODO(), req)
assert.NoError(t, err) assert.NoError(t, err)
assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
rateCol.Register(metricsinfo.InsertConsumeThroughput) util.RegisterRateCollector(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)
}) })
} }

View File

@ -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()
}
}

View File

@ -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)
}

View File

@ -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)
})
}

View File

@ -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
}

View File

@ -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))
}

View File

@ -20,6 +20,7 @@ import (
"context" "context"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "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/hardware"
"github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/metricsinfo"
@ -33,7 +34,7 @@ func (node *DataNode) getQuotaMetrics() (*metricsinfo.DataNodeQuotaMetrics, erro
var err error var err error
rms := make([]metricsinfo.RateMetric, 0) rms := make([]metricsinfo.RateMetric, 0)
getRateMetric := func(label metricsinfo.RateMetricLabel) { getRateMetric := func(label metricsinfo.RateMetricLabel) {
rate, err2 := rateCol.Rate(label, ratelimitutil.DefaultAvgDuration) rate, err2 := util.RateCol.Rate(label, ratelimitutil.DefaultAvgDuration)
if err2 != nil { if err2 != nil {
err = err2 err = err2
return return
@ -49,7 +50,7 @@ func (node *DataNode) getQuotaMetrics() (*metricsinfo.DataNodeQuotaMetrics, erro
return nil, err return nil, err
} }
minFGChannel, minFGTt := rateCol.getMinFlowGraphTt() minFGChannel, minFGTt := util.RateCol.GetMinFlowGraphTt()
return &metricsinfo.DataNodeQuotaMetrics{ return &metricsinfo.DataNodeQuotaMetrics{
Hms: metricsinfo.HardwareMetrics{}, Hms: metricsinfo.HardwareMetrics{},
Rms: rms, Rms: rms,

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package pipeline
import ( import (
"context" "context"
@ -23,7 +23,6 @@ import (
"go.uber.org/zap" "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/broker"
"github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/datanode/compaction"
"github.com/milvus-io/milvus/internal/datanode/io" "github.com/milvus-io/milvus/internal/datanode/io"
@ -45,33 +44,27 @@ import (
"github.com/milvus-io/milvus/pkg/util/typeutil" "github.com/milvus-io/milvus/pkg/util/typeutil"
) )
// dataSyncService controls a flowgraph for a specific collection // DataSyncService controls a flowgraph for a specific collection
type dataSyncService struct { type DataSyncService struct {
ctx context.Context ctx context.Context
cancelFn context.CancelFunc cancelFn context.CancelFunc
metacache metacache.MetaCache metacache metacache.MetaCache
opID int64 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 vchannelName string
// TODO: should be equal to paramtable.GetNodeID(), but intergrationtest has 1 paramtable for a minicluster, the NodeID // 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. // varies, will cause savebinglogpath check fail. So we pass ServerID into DataSyncService to aviod it failure.
serverID UniqueID serverID util.UniqueID
fg *flowgraph.TimeTickedFlowGraph // internal flowgraph processes insert/delta messages fg *flowgraph.TimeTickedFlowGraph // internal flowgraph processes insert/delta messages
broker broker.Broker broker broker.Broker
syncMgr syncmgr.SyncManager syncMgr syncmgr.SyncManager
flushCh chan flushMsg timetickSender *util.TimeTickSender // reference to TimeTickSender
resendTTCh chan resendTTMsg // chan to ask for resending DataNode time tick message. compactor compaction.Executor // reference to compaction executor
timetickSender *timeTickSender // reference to timeTickSender
compactor compaction.Executor // reference to compaction executor
flushingSegCache *Cache // a guarding cache stores currently flushing segment ids
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 dispClient msgdispatcher.Client
chunkManager storage.ChunkManager chunkManager storage.ChunkManager
@ -80,15 +73,14 @@ type dataSyncService struct {
type nodeConfig struct { type nodeConfig struct {
msFactory msgstream.Factory // msgStream factory msFactory msgstream.Factory // msgStream factory
collectionID UniqueID collectionID util.UniqueID
vChannelName string vChannelName string
metacache metacache.MetaCache metacache metacache.MetaCache
allocator allocator.Allocator serverID util.UniqueID
serverID UniqueID
} }
// start the flow graph in dataSyncService // Start the flow graph in dataSyncService
func (dsService *dataSyncService) start() { func (dsService *DataSyncService) Start() {
if dsService.fg != nil { if dsService.fg != nil {
log.Info("dataSyncService starting flow graph", zap.Int64("collectionID", dsService.collectionID), log.Info("dataSyncService starting flow graph", zap.Int64("collectionID", dsService.collectionID),
zap.String("vChanName", dsService.vchannelName)) 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 { if dsService.fg != nil {
log.Info("dataSyncService gracefully closing flowgraph") log.Info("dataSyncService gracefully closing flowgraph")
dsService.fg.SetCloseMethod(flowgraph.CloseGracefully) 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() { dsService.stopOnce.Do(func() {
log := log.Ctx(dsService.ctx).With( log := log.Ctx(dsService.ctx).With(
zap.Int64("collectionID", dsService.collectionID), 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) { func (dsService *DataSyncService) GetMetaCache() metacache.MetaCache {
tickler.setTotal(int32(len(unflushed) + len(flushed))) return dsService.metacache
return initMetaCache(initCtx, storageV2Cache, node.chunkManager, info, tickler, unflushed, flushed)
} }
func getMetaCacheWithEtcdTickler(initCtx context.Context, node *DataNode, info *datapb.ChannelWatchInfo, tickler *etcdTickler, unflushed, flushed []*datapb.SegmentInfo, storageV2Cache *metacache.StorageV2Cache) (metacache.MetaCache, error) { 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.watch() tickler.SetTotal(int32(len(unflushed) + len(flushed)))
defer tickler.stop() return initMetaCache(initCtx, storageV2Cache, params.ChunkManager, info, tickler, unflushed, flushed)
return initMetaCache(initCtx, storageV2Cache, node.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 // tickler will update addSegment progress to watchInfo
futures := make([]*conc.Future[any], 0, len(unflushed)+len(flushed)) futures := make([]*conc.Future[any], 0, len(unflushed)+len(flushed))
segmentPks := typeutil.NewConcurrentMap[int64, []*storage.PkStatistics]() segmentPks := typeutil.NewConcurrentMap[int64, []*storage.PkStatistics]()
@ -161,15 +154,15 @@ func initMetaCache(initCtx context.Context, storageV2Cache *metacache.StorageV2C
var stats []*storage.PkStatistics var stats []*storage.PkStatistics
var err error var err error
if params.Params.CommonCfg.EnableStorageV2.GetAsBool() { if params.Params.CommonCfg.EnableStorageV2.GetAsBool() {
stats, err = util.LoadStatsV2(storageV2Cache, segment, info.GetSchema()) stats, err = compaction.LoadStatsV2(storageV2Cache, segment, info.GetSchema())
} else { } 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 { if err != nil {
return nil, err return nil, err
} }
segmentPks.Insert(segment.GetID(), stats) segmentPks.Insert(segment.GetID(), stats)
tickler.inc() tickler.Inc()
return struct{}{}, nil return struct{}{}, nil
}) })
@ -198,88 +191,77 @@ func initMetaCache(initCtx context.Context, storageV2Cache *metacache.StorageV2C
return metacache, nil 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 ( var (
channelName = info.GetVchan().GetChannelName() channelName = info.GetVchan().GetChannelName()
collectionID = info.GetVchan().GetCollectionID() collectionID = info.GetVchan().GetCollectionID()
) )
config := &nodeConfig{ config := &nodeConfig{
msFactory: node.factory, msFactory: params.MsgStreamFactory,
allocator: node.allocator,
collectionID: collectionID, collectionID: collectionID,
vChannelName: channelName, vChannelName: channelName,
metacache: metacache, metacache: metacache,
serverID: node.session.ServerID, serverID: params.Session.ServerID,
} }
var ( err := params.WriteBufferManager.Register(channelName, metacache, storageV2Cache,
flushCh = make(chan flushMsg, 100) writebuffer.WithMetaWriter(syncmgr.BrokerMetaWriter(params.Broker, config.serverID)),
resendTTCh = make(chan resendTTMsg, 100) writebuffer.WithIDAllocator(params.Allocator))
)
err := node.writeBufferManager.Register(channelName, metacache, storageV2Cache, writebuffer.WithMetaWriter(syncmgr.BrokerMetaWriter(node.broker, config.serverID)), writebuffer.WithIDAllocator(node.allocator))
if err != nil { if err != nil {
log.Warn("failed to register channel buffer", zap.Error(err)) log.Warn("failed to register channel buffer", zap.Error(err))
return nil, err return nil, err
} }
defer func() { defer func() {
if err != nil { if err != nil {
defer node.writeBufferManager.RemoveChannel(channelName) defer params.WriteBufferManager.RemoveChannel(channelName)
} }
}() }()
ctx, cancel := context.WithCancel(node.ctx) ctx, cancel := context.WithCancel(params.Ctx)
ds := &dataSyncService{ ds := &DataSyncService{
ctx: ctx, ctx: ctx,
cancelFn: cancel, cancelFn: cancel,
flushCh: flushCh, opID: info.GetOpID(),
resendTTCh: resendTTCh,
opID: info.GetOpID(),
dispClient: node.dispClient, dispClient: params.DispClient,
msFactory: node.factory, broker: params.Broker,
broker: node.broker,
idAllocator: config.allocator,
metacache: config.metacache, metacache: config.metacache,
collectionID: config.collectionID, collectionID: config.collectionID,
vchannelName: config.vChannelName, vchannelName: config.vChannelName,
serverID: config.serverID, serverID: config.serverID,
flushingSegCache: node.segmentCache, chunkManager: params.ChunkManager,
clearSignal: node.clearSignal, compactor: params.CompactionExecutor,
chunkManager: node.chunkManager, timetickSender: params.TimeTickSender,
compactor: node.compactionExecutor, syncMgr: params.SyncMgr,
timetickSender: node.timeTickSender,
syncMgr: node.syncMgr,
fg: nil, fg: nil,
} }
// init flowgraph // init flowgraph
fg := flowgraph.NewTimeTickedFlowGraph(node.ctx) fg := flowgraph.NewTimeTickedFlowGraph(params.Ctx)
dmStreamNode, err := newDmInputNode(initCtx, node.dispClient, info.GetVchan().GetSeekPosition(), config) dmStreamNode, err := newDmInputNode(initCtx, params.DispClient, info.GetVchan().GetSeekPosition(), config)
if err != nil { if err != nil {
return nil, err return nil, err
} }
ddNode, err := newDDNode( ddNode, err := newDDNode(
node.ctx, params.Ctx,
collectionID, collectionID,
channelName, channelName,
info.GetVchan().GetDroppedSegmentIds(), info.GetVchan().GetDroppedSegmentIds(),
flushed, flushed,
unflushed, unflushed,
node.compactionExecutor, params.CompactionExecutor,
) )
if err != nil { if err != nil {
return nil, err return nil, err
} }
var updater statsUpdater var updater statsUpdater
if Params.DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() { if paramtable.Get().DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() {
updater = ds.timetickSender updater = ds.timetickSender
} else { } else {
m, err := config.msFactory.NewMsgStream(ctx) m, err := config.msFactory.NewMsgStream(ctx)
@ -287,18 +269,18 @@ func getServiceWithChannel(initCtx context.Context, node *DataNode, info *datapb
return nil, err 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() 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) m.EnableProduce(true)
updater = newMqStatsUpdater(config, m) 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 { if err != nil {
return nil, err return nil, err
} }
@ -311,16 +293,16 @@ func getServiceWithChannel(initCtx context.Context, node *DataNode, info *datapb
return ds, nil 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 // initCtx is used to init the dataSyncService only, if initCtx.Canceled or initCtx.Timeout
// newServiceWithEtcdTickler stops and returns the initCtx.Err() // NewDataSyncService stops and returns the initCtx.Err()
func newServiceWithEtcdTickler(initCtx context.Context, node *DataNode, info *datapb.ChannelWatchInfo, tickler *etcdTickler) (*dataSyncService, error) { func NewDataSyncService(initCtx context.Context, pipelineParams *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler) (*DataSyncService, error) {
// recover segment checkpoints // recover segment checkpoints
unflushedSegmentInfos, err := node.broker.GetSegmentInfo(initCtx, info.GetVchan().GetUnflushedSegmentIds()) unflushedSegmentInfos, err := pipelineParams.Broker.GetSegmentInfo(initCtx, info.GetVchan().GetUnflushedSegmentIds())
if err != nil { if err != nil {
return nil, err return nil, err
} }
flushedSegmentInfos, err := node.broker.GetSegmentInfo(initCtx, info.GetVchan().GetFlushedSegmentIds()) flushedSegmentInfos, err := pipelineParams.Broker.GetSegmentInfo(initCtx, info.GetVchan().GetFlushedSegmentIds())
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -332,42 +314,16 @@ func newServiceWithEtcdTickler(initCtx context.Context, node *DataNode, info *da
return nil, err 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 // 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 { if err != nil {
return nil, err 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}
} }

View File

@ -14,12 +14,10 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package pipeline
import ( import (
"bytes"
"context" "context"
"encoding/binary"
"fmt" "fmt"
"math" "math"
"math/rand" "math/rand"
@ -29,7 +27,6 @@ import (
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock" "github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite" "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/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "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/allocator"
"github.com/milvus-io/milvus/internal/datanode/broker" "github.com/milvus-io/milvus/internal/datanode/broker"
"github.com/milvus-io/milvus/internal/datanode/metacache" "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/datanode/writebuffer"
"github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/proto/datapb" "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/dependency"
"github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/pkg/common" "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/msgdispatcher"
"github.com/milvus-io/milvus/pkg/mq/msgstream" "github.com/milvus-io/milvus/pkg/mq/msgstream"
"github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/paramtable"
@ -54,10 +52,6 @@ import (
var dataSyncServiceTestDir = "/tmp/milvus_test/data_sync_service" var dataSyncServiceTestDir = "/tmp/milvus_test/data_sync_service"
func init() {
paramtable.Init()
}
func getWatchInfo(info *testInfo) *datapb.ChannelWatchInfo { func getWatchInfo(info *testInfo) *datapb.ChannelWatchInfo {
return &datapb.ChannelWatchInfo{ return &datapb.ChannelWatchInfo{
Vchan: getVchanInfo(info), Vchan: getVchanInfo(info),
@ -131,16 +125,16 @@ type testInfo struct {
channelNil bool channelNil bool
inMsgFactory dependency.Factory inMsgFactory dependency.Factory
collID UniqueID collID util.UniqueID
chanName string chanName string
ufCollID UniqueID ufCollID util.UniqueID
ufSegID UniqueID ufSegID util.UniqueID
ufchanName string ufchanName string
ufNor int64 ufNor int64
fCollID UniqueID fCollID util.UniqueID
fSegID UniqueID fSegID util.UniqueID
fchanName string fchanName string
fNor int64 fNor int64
@ -176,18 +170,55 @@ func TestDataSyncService_newDataSyncService(t *testing.T) {
cm := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir)) cm := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir))
defer cm.RemoveWithPrefix(ctx, cm.RootPath()) defer cm.RemoveWithPrefix(ctx, cm.RootPath())
node := newIDLEDataNodeMock(ctx, schemapb.DataType_Int64) wbManager := writebuffer.NewMockBufferManager(t)
node.allocator = allocator.NewMockAllocator(t) wbManager.EXPECT().
Register(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil)
for _, test := range tests { for _, test := range tests {
t.Run(test.description, func(t *testing.T) { t.Run(test.description, func(t *testing.T) {
node.factory = test.inMsgFactory mockBroker := broker.NewMockBroker(t)
defer node.tryToReleaseFlowgraph(test.chanName) mockBroker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).Call.Return(
ds, err := newServiceWithEtcdTickler( 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, ctx,
node, pipelineParams,
getWatchInfo(test), getWatchInfo(test),
genTestTickler(), util.NewTickler(),
) )
if !test.isValidCase { if !test.isValidCase {
@ -199,122 +230,31 @@ func TestDataSyncService_newDataSyncService(t *testing.T) {
// start // start
ds.fg = nil 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) { func TestGetChannelWithTickler(t *testing.T) {
channelName := "by-dev-rootcoord-dml-0" channelName := "by-dev-rootcoord-dml-0"
info := getWatchInfoByOpID(100, channelName, datapb.ChannelWatchState_ToWatch) info := util.GetWatchInfoByOpID(100, channelName, datapb.ChannelWatchState_ToWatch)
node := newIDLEDataNodeMock(context.Background(), schemapb.DataType_Int64) chunkManager := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir))
node.chunkManager = storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir)) defer chunkManager.RemoveWithPrefix(context.Background(), chunkManager.RootPath())
defer node.chunkManager.RemoveWithPrefix(context.Background(), node.chunkManager.RootPath())
meta := NewMetaFactory().GetCollectionMeta(1, "test_collection", schemapb.DataType_Int64) meta := util.NewMetaFactory().GetCollectionMeta(1, "test_collection", schemapb.DataType_Int64)
broker := broker.NewMockBroker(t)
node.broker = broker
info.Schema = meta.GetSchema() 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{ unflushed := []*datapb.SegmentInfo{
{ {
ID: 100, 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.NoError(t, err)
assert.NotNil(t, metaCache) assert.NotNil(t, metaCache)
assert.Equal(t, int64(1), metaCache.Collection()) assert.Equal(t, int64(1), metaCache.Collection())
@ -359,49 +299,39 @@ func TestGetChannelWithTickler(t *testing.T) {
type DataSyncServiceSuite struct { type DataSyncServiceSuite struct {
suite.Suite suite.Suite
MockDataSuiteBase util.MockDataSuiteBase
node *DataNode // node param pipelineParams *util.PipelineParams // node param
chunkManager *mocks.ChunkManager chunkManager *mocks.ChunkManager
broker *broker.MockBroker broker *broker.MockBroker
allocator *allocator.MockAllocator allocator *allocator.MockAllocator
wbManager *writebuffer.MockBufferManager wbManager *writebuffer.MockBufferManager
channelCheckpointUpdater *util.ChannelCheckpointUpdater
factory *dependency.MockFactory factory *dependency.MockFactory
ms *msgstream.MockMsgStream ms *msgstream.MockMsgStream
msChan chan *msgstream.MsgPack msChan chan *msgstream.MsgPack
} }
func (s *DataSyncServiceSuite) SetupSuite() { func (s *DataSyncServiceSuite) SetupSuite() {
paramtable.Get().Init(paramtable.NewBaseTable()) paramtable.Get().Init(paramtable.NewBaseTable())
s.MockDataSuiteBase.prepareData() s.MockDataSuiteBase.PrepareData()
} }
func (s *DataSyncServiceSuite) SetupTest() { func (s *DataSyncServiceSuite) SetupTest() {
s.node = &DataNode{}
s.chunkManager = mocks.NewChunkManager(s.T()) s.chunkManager = mocks.NewChunkManager(s.T())
s.broker = broker.NewMockBroker(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.allocator = allocator.NewMockAllocator(s.T())
s.wbManager = writebuffer.NewMockBufferManager(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") paramtable.Get().Save(paramtable.Get().DataNodeCfg.ChannelCheckpointUpdateTickInSeconds.Key, "0.01")
defer paramtable.Get().Save(paramtable.Get().DataNodeCfg.ChannelCheckpointUpdateTickInSeconds.Key, "10") defer paramtable.Get().Save(paramtable.Get().DataNodeCfg.ChannelCheckpointUpdateTickInSeconds.Key, "10")
go s.node.channelCheckpointUpdater.start() s.channelCheckpointUpdater = util.NewChannelCheckpointUpdater(s.broker)
s.msChan = make(chan *msgstream.MsgPack)
go s.channelCheckpointUpdater.Start()
s.msChan = make(chan *msgstream.MsgPack, 1)
s.factory = dependency.NewMockFactory(s.T()) s.factory = dependency.NewMockFactory(s.T())
s.ms = msgstream.NewMockMsgStream(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().Chan().Return(s.msChan)
s.ms.EXPECT().Close().Return() s.ms.EXPECT().Close().Return()
s.node.factory = s.factory s.pipelineParams = &util.PipelineParams{
s.node.dispClient = msgdispatcher.NewClient(s.factory, typeutil.DataNodeRole, 1) Ctx: context.TODO(),
MsgStreamFactory: s.factory,
s.node.timeTickSender = newTimeTickSender(s.broker, 0) 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() { func (s *DataSyncServiceSuite) TestStartStop() {
var ( var (
insertChannelName = fmt.Sprintf("by-dev-rootcoord-dml-%d", rand.Int()) insertChannelName = fmt.Sprintf("by-dev-rootcoord-dml-%d", rand.Int())
Factory = &MetaFactory{} Factory = &util.MetaFactory{}
collMeta = Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64) collMeta = Factory.GetCollectionMeta(util.UniqueID(0), "coll1", schemapb.DataType_Int64)
) )
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
defer cancel() defer cancel()
@ -486,30 +425,30 @@ func (s *DataSyncServiceSuite) TestStartStop() {
}, },
} }
sync, err := newServiceWithEtcdTickler( sync, err := NewDataSyncService(
ctx, ctx,
s.node, s.pipelineParams,
watchInfo, watchInfo,
genTestTickler(), util.NewTickler(),
) )
s.Require().NoError(err) s.Require().NoError(err)
s.Require().NotNil(sync) s.Require().NotNil(sync)
sync.start() sync.Start()
defer sync.close() defer sync.close()
timeRange := TimeRange{ timeRange := util.TimeRange{
timestampMin: 0, TimestampMin: 0,
timestampMax: math.MaxUint64 - 1, TimestampMax: math.MaxUint64 - 1,
} }
msgTs := tsoutil.GetCurrentTime() msgTs := tsoutil.GetCurrentTime()
dataFactory := NewDataFactory() dataFactory := util.NewDataFactory()
insertMessages := dataFactory.GetMsgStreamTsInsertMsgs(2, insertChannelName, msgTs) insertMessages := dataFactory.GetMsgStreamTsInsertMsgs(2, insertChannelName, msgTs)
msgPack := msgstream.MsgPack{ msgPack := msgstream.MsgPack{
BeginTs: timeRange.timestampMin, BeginTs: timeRange.TimestampMin,
EndTs: timeRange.timestampMax, EndTs: timeRange.TimestampMax,
Msgs: insertMessages, Msgs: insertMessages,
StartPositions: []*msgpb.MsgPosition{{ StartPositions: []*msgpb.MsgPosition{{
Timestamp: msgTs, Timestamp: msgTs,
@ -533,7 +472,7 @@ func (s *DataSyncServiceSuite) TestStartStop() {
TimeTickMsg: msgpb.TimeTickMsg{ TimeTickMsg: msgpb.TimeTickMsg{
Base: &commonpb.MsgBase{ Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_TimeTick, MsgType: commonpb.MsgType_TimeTick,
MsgID: UniqueID(0), MsgID: util.UniqueID(0),
Timestamp: tsoutil.GetCurrentTime(), Timestamp: tsoutil.GetCurrentTime(),
SourceID: 0, SourceID: 0,
}, },
@ -546,7 +485,6 @@ func (s *DataSyncServiceSuite) TestStartStop() {
s.wbManager.EXPECT().NotifyCheckpointUpdated(insertChannelName, msgTs).Return().Maybe() s.wbManager.EXPECT().NotifyCheckpointUpdated(insertChannelName, msgTs).Return().Maybe()
ch := make(chan struct{}) ch := make(chan struct{})
s.broker.EXPECT().UpdateChannelCheckpoint(mock.Anything, mock.Anything).RunAndReturn(func(_ context.Context, _ []*msgpb.MsgPosition) error { s.broker.EXPECT().UpdateChannelCheckpoint(mock.Anything, mock.Anything).RunAndReturn(func(_ context.Context, _ []*msgpb.MsgPosition) error {
close(ch) close(ch)
return nil return nil

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package pipeline
import ( import (
"context" "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/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "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/compaction"
"github.com/milvus-io/milvus/internal/datanode/util"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/internal/util/flowgraph"
"github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/log"
@ -61,15 +62,15 @@ type ddNode struct {
BaseNode BaseNode
ctx context.Context ctx context.Context
collectionID UniqueID collectionID util.UniqueID
vChannelName string vChannelName string
dropMode atomic.Value dropMode atomic.Value
compactionExecutor compaction.Executor compactionExecutor compaction.Executor
// for recovery // for recovery
growingSegInfo map[UniqueID]*datapb.SegmentInfo // segmentID growingSegInfo map[util.UniqueID]*datapb.SegmentInfo // segmentID
sealedSegInfo map[UniqueID]*datapb.SegmentInfo // segmentID sealedSegInfo map[util.UniqueID]*datapb.SegmentInfo // segmentID
droppedSegmentIDs []int64 droppedSegmentIDs []int64
} }
@ -99,15 +100,15 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
} }
if msMsg.IsCloseMsg() { if msMsg.IsCloseMsg() {
fgMsg := flowGraphMsg{ fgMsg := FlowGraphMsg{
BaseMsg: flowgraph.NewBaseMsg(true), BaseMsg: flowgraph.NewBaseMsg(true),
insertMessages: make([]*msgstream.InsertMsg, 0), InsertMessages: make([]*msgstream.InsertMsg, 0),
timeRange: TimeRange{ TimeRange: util.TimeRange{
timestampMin: msMsg.TimestampMin(), TimestampMin: msMsg.TimestampMin(),
timestampMax: msMsg.TimestampMax(), TimestampMax: msMsg.TimestampMax(),
}, },
startPositions: msMsg.StartPositions(), StartPositions: msMsg.StartPositions(),
endPositions: msMsg.EndPositions(), EndPositions: msMsg.EndPositions(),
dropCollection: false, dropCollection: false,
} }
log.Warn("MsgStream closed", zap.Any("ddNode node", ddn.Name()), zap.String("channel", ddn.vChannelName), zap.Int64("collection", ddn.collectionID)) 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 var spans []trace.Span
for _, msg := range msMsg.TsMessages() { for _, msg := range msMsg.TsMessages() {
ctx, sp := startTracer(msg, "DDNode-Operate") ctx, sp := util.StartTracer(msg, "DDNode-Operate")
spans = append(spans, sp) spans = append(spans, sp)
msg.SetTraceCtx(ctx) msg.SetTraceCtx(ctx)
} }
@ -131,14 +132,14 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
} }
}() }()
fgMsg := flowGraphMsg{ fgMsg := FlowGraphMsg{
insertMessages: make([]*msgstream.InsertMsg, 0), InsertMessages: make([]*msgstream.InsertMsg, 0),
timeRange: TimeRange{ TimeRange: util.TimeRange{
timestampMin: msMsg.TimestampMin(), TimestampMin: msMsg.TimestampMin(),
timestampMax: msMsg.TimestampMax(), TimestampMax: msMsg.TimestampMax(),
}, },
startPositions: make([]*msgpb.MsgPosition, 0), StartPositions: make([]*msgpb.MsgPosition, 0),
endPositions: make([]*msgpb.MsgPosition, 0), EndPositions: make([]*msgpb.MsgPosition, 0),
dropCollection: false, dropCollection: false,
} }
@ -180,7 +181,7 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
continue continue
} }
rateCol.Add(metricsinfo.InsertConsumeThroughput, float64(proto.Size(&imsg.InsertRequest))) util.RateCol.Add(metricsinfo.InsertConsumeThroughput, float64(proto.Size(&imsg.InsertRequest)))
metrics.DataNodeConsumeBytesCount. metrics.DataNodeConsumeBytesCount.
WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.InsertLabel). WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.InsertLabel).
@ -198,7 +199,7 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
zap.Int64("segmentID", imsg.GetSegmentID()), zap.Int64("segmentID", imsg.GetSegmentID()),
zap.String("channel", ddn.vChannelName), zap.String("channel", ddn.vChannelName),
zap.Int("numRows", len(imsg.GetRowIDs()))) zap.Int("numRows", len(imsg.GetRowIDs())))
fgMsg.insertMessages = append(fgMsg.insertMessages, imsg) fgMsg.InsertMessages = append(fgMsg.InsertMessages, imsg)
case commonpb.MsgType_Delete: case commonpb.MsgType_Delete:
dmsg := msg.(*msgstream.DeleteMsg) 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)) 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. metrics.DataNodeConsumeBytesCount.
WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.DeleteLabel). WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.DeleteLabel).
@ -225,12 +226,12 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
metrics.DataNodeConsumeMsgRowsCount. metrics.DataNodeConsumeMsgRowsCount.
WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.DeleteLabel). WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.DeleteLabel).
Add(float64(dmsg.GetNumRows())) Add(float64(dmsg.GetNumRows()))
fgMsg.deleteMessages = append(fgMsg.deleteMessages, dmsg) fgMsg.DeleteMessages = append(fgMsg.DeleteMessages, dmsg)
} }
} }
fgMsg.startPositions = append(fgMsg.startPositions, msMsg.StartPositions()...) fgMsg.StartPositions = append(fgMsg.StartPositions, msMsg.StartPositions()...)
fgMsg.endPositions = append(fgMsg.endPositions, msMsg.EndPositions()...) fgMsg.EndPositions = append(fgMsg.EndPositions, msMsg.EndPositions()...)
return []Msg{&fgMsg} return []Msg{&fgMsg}
} }
@ -267,7 +268,7 @@ func (ddn *ddNode) tryToFilterSegmentInsertMessages(msg *msgstream.InsertMsg) bo
return false return false
} }
func (ddn *ddNode) isDropped(segID UniqueID) bool { func (ddn *ddNode) isDropped(segID util.UniqueID) bool {
for _, droppedSegmentID := range ddn.droppedSegmentIDs { for _, droppedSegmentID := range ddn.droppedSegmentIDs {
if droppedSegmentID == segID { if droppedSegmentID == segID {
return true return true
@ -280,19 +281,19 @@ func (ddn *ddNode) Close() {
log.Info("Flowgraph DD Node closing") 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, sealedSegments []*datapb.SegmentInfo, growingSegments []*datapb.SegmentInfo, executor compaction.Executor,
) (*ddNode, error) { ) (*ddNode, error) {
baseNode := BaseNode{} baseNode := BaseNode{}
baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32()) baseNode.SetMaxQueueLength(paramtable.Get().DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32())
baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32()) baseNode.SetMaxParallelism(paramtable.Get().DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32())
dd := &ddNode{ dd := &ddNode{
ctx: ctx, ctx: ctx,
BaseNode: baseNode, BaseNode: baseNode,
collectionID: collID, collectionID: collID,
sealedSegInfo: make(map[UniqueID]*datapb.SegmentInfo, len(sealedSegments)), sealedSegInfo: make(map[util.UniqueID]*datapb.SegmentInfo, len(sealedSegments)),
growingSegInfo: make(map[UniqueID]*datapb.SegmentInfo, len(growingSegments)), growingSegInfo: make(map[util.UniqueID]*datapb.SegmentInfo, len(growingSegments)),
droppedSegmentIDs: droppedSegmentIDs, droppedSegmentIDs: droppedSegmentIDs,
vChannelName: vChannelName, vChannelName: vChannelName,
compactionExecutor: executor, compactionExecutor: executor,

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package pipeline
import ( import (
"context" "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/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "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/compaction"
"github.com/milvus-io/milvus/internal/datanode/util"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/internal/util/flowgraph"
"github.com/milvus-io/milvus/pkg/mq/msgstream" "github.com/milvus-io/milvus/pkg/mq/msgstream"
@ -63,9 +64,9 @@ func TestFlowGraph_DDNode_newDDNode(t *testing.T) {
} }
var ( var (
collectionID = UniqueID(1) collectionID = util.UniqueID(1)
channelName = fmt.Sprintf("by-dev-rootcoord-dml-%s", t.Name()) channelName = fmt.Sprintf("by-dev-rootcoord-dml-%s", t.Name())
droppedSegIDs = []UniqueID{} droppedSegIDs = []util.UniqueID{}
) )
for _, test := range tests { for _, test := range tests {
@ -102,11 +103,11 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) {
"Invalid input length == 0", "Invalid input length == 0",
}, },
{ {
[]Msg{&flowGraphMsg{}, &flowGraphMsg{}, &flowGraphMsg{}}, []Msg{&FlowGraphMsg{}, &FlowGraphMsg{}, &FlowGraphMsg{}},
"Invalid input length == 3", "Invalid input length == 3",
}, },
{ {
[]Msg{&flowGraphMsg{}}, []Msg{&FlowGraphMsg{}},
"Invalid input length == 1 but input message is not msgStreamMsg", "Invalid input length == 1 but input message is not msgStreamMsg",
}, },
} }
@ -119,9 +120,9 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) {
} }
// valid inputs // valid inputs
tests := []struct { tests := []struct {
ddnCollID UniqueID ddnCollID util.UniqueID
msgCollID UniqueID msgCollID util.UniqueID
expectedChlen int expectedChlen int
description string description string
@ -158,7 +159,7 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) {
if test.ddnCollID == test.msgCollID { if test.ddnCollID == test.msgCollID {
assert.NotEmpty(t, rt) assert.NotEmpty(t, rt)
assert.True(t, rt[0].(*flowGraphMsg).dropCollection) assert.True(t, rt[0].(*FlowGraphMsg).dropCollection)
} else { } else {
assert.NotEmpty(t, rt) 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) { t.Run("Test DDNode Operate DropPartition Msg", func(t *testing.T) {
// valid inputs // valid inputs
tests := []struct { tests := []struct {
ddnCollID UniqueID ddnCollID util.UniqueID
msgCollID UniqueID msgCollID util.UniqueID
msgPartID UniqueID msgPartID util.UniqueID
expectOutput []UniqueID expectOutput []util.UniqueID
description string description string
}{ }{
{ {
1, 1, 101, 1, 1, 101,
[]UniqueID{101}, []util.UniqueID{101},
"DropCollectionMsg collID == ddNode collID", "DropCollectionMsg collID == ddNode collID",
}, },
{ {
1, 2, 101, 1, 2, 101,
[]UniqueID{}, []util.UniqueID{},
"DropCollectionMsg collID != ddNode collID", "DropCollectionMsg collID != ddNode collID",
}, },
} }
@ -211,7 +212,7 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) {
rt := ddn.Operate([]Msg{msgStreamMsg}) rt := ddn.Operate([]Msg{msgStreamMsg})
assert.NotEmpty(t, rt) assert.NotEmpty(t, rt)
fgMsg, ok := rt[0].(*flowGraphMsg) fgMsg, ok := rt[0].(*FlowGraphMsg)
assert.True(t, ok) assert.True(t, ok)
assert.ElementsMatch(t, test.expectOutput, fgMsg.dropPartitions) 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) { 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 // Prepare ddNode states
ddn := ddNode{ ddn := ddNode{
ctx: context.Background(), ctx: context.Background(),
collectionID: collectionID, collectionID: collectionID,
droppedSegmentIDs: []UniqueID{100}, droppedSegmentIDs: []util.UniqueID{100},
} }
tsMessages := []msgstream.TsMsg{getInsertMsg(100, 10000), getInsertMsg(200, 20000)} tsMessages := []msgstream.TsMsg{getInsertMsg(100, 10000), getInsertMsg(200, 20000)}
var msgStreamMsg Msg = flowgraph.GenerateMsgStreamMsg(tsMessages, 0, 0, nil, nil) var msgStreamMsg Msg = flowgraph.GenerateMsgStreamMsg(tsMessages, 0, 0, nil, nil)
rt := ddn.Operate([]Msg{msgStreamMsg}) 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) { t.Run("Test DDNode Operate Delete Msg", func(t *testing.T) {
tests := []struct { tests := []struct {
ddnCollID UniqueID ddnCollID util.UniqueID
inMsgCollID UniqueID inMsgCollID util.UniqueID
MsgEndTs Timestamp MsgEndTs util.Timestamp
expectedRtLen int expectedRtLen int
description string description string
@ -272,7 +273,7 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) {
// Test // Test
rt := ddn.Operate([]Msg{msgStreamMsg}) 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 { tests := []struct {
description string description string
droppedSegIDs []UniqueID droppedSegIDs []util.UniqueID
sealedSegInfo map[UniqueID]*datapb.SegmentInfo sealedSegInfo map[util.UniqueID]*datapb.SegmentInfo
growingSegInfo map[UniqueID]*datapb.SegmentInfo growingSegInfo map[util.UniqueID]*datapb.SegmentInfo
inMsg *msgstream.InsertMsg inMsg *msgstream.InsertMsg
expected bool expected bool
}{ }{
{ {
"test dropped segments true", "test dropped segments true",
[]UniqueID{100}, []util.UniqueID{100},
nil, nil,
nil, nil,
getInsertMsg(100, 10000), getInsertMsg(100, 10000),
@ -299,7 +300,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
}, },
{ {
"test dropped segments true 2", "test dropped segments true 2",
[]UniqueID{100, 101, 102}, []util.UniqueID{100, 101, 102},
nil, nil,
nil, nil,
getInsertMsg(102, 10000), getInsertMsg(102, 10000),
@ -307,8 +308,8 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
}, },
{ {
"test sealed segments msgTs <= segmentTs true", "test sealed segments msgTs <= segmentTs true",
[]UniqueID{}, []util.UniqueID{},
map[UniqueID]*datapb.SegmentInfo{ map[util.UniqueID]*datapb.SegmentInfo{
200: getSegmentInfo(200, 50000), 200: getSegmentInfo(200, 50000),
300: getSegmentInfo(300, 50000), 300: getSegmentInfo(300, 50000),
}, },
@ -318,8 +319,8 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
}, },
{ {
"test sealed segments msgTs <= segmentTs true", "test sealed segments msgTs <= segmentTs true",
[]UniqueID{}, []util.UniqueID{},
map[UniqueID]*datapb.SegmentInfo{ map[util.UniqueID]*datapb.SegmentInfo{
200: getSegmentInfo(200, 50000), 200: getSegmentInfo(200, 50000),
300: getSegmentInfo(300, 50000), 300: getSegmentInfo(300, 50000),
}, },
@ -329,8 +330,8 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
}, },
{ {
"test sealed segments msgTs > segmentTs false", "test sealed segments msgTs > segmentTs false",
[]UniqueID{}, []util.UniqueID{},
map[UniqueID]*datapb.SegmentInfo{ map[util.UniqueID]*datapb.SegmentInfo{
200: getSegmentInfo(200, 50000), 200: getSegmentInfo(200, 50000),
300: getSegmentInfo(300, 50000), 300: getSegmentInfo(300, 50000),
}, },
@ -340,9 +341,9 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
}, },
{ {
"test growing segments msgTs <= segmentTs true", "test growing segments msgTs <= segmentTs true",
[]UniqueID{}, []util.UniqueID{},
nil, nil,
map[UniqueID]*datapb.SegmentInfo{ map[util.UniqueID]*datapb.SegmentInfo{
200: getSegmentInfo(200, 50000), 200: getSegmentInfo(200, 50000),
300: getSegmentInfo(300, 50000), 300: getSegmentInfo(300, 50000),
}, },
@ -351,9 +352,9 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
}, },
{ {
"test growing segments msgTs > segmentTs false", "test growing segments msgTs > segmentTs false",
[]UniqueID{}, []util.UniqueID{},
nil, nil,
map[UniqueID]*datapb.SegmentInfo{ map[util.UniqueID]*datapb.SegmentInfo{
200: getSegmentInfo(200, 50000), 200: getSegmentInfo(200, 50000),
300: getSegmentInfo(300, 50000), 300: getSegmentInfo(300, 50000),
}, },
@ -362,12 +363,12 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
}, },
{ {
"test not exist", "test not exist",
[]UniqueID{}, []util.UniqueID{},
map[UniqueID]*datapb.SegmentInfo{ map[util.UniqueID]*datapb.SegmentInfo{
400: getSegmentInfo(500, 50000), 400: getSegmentInfo(500, 50000),
500: getSegmentInfo(400, 50000), 500: getSegmentInfo(400, 50000),
}, },
map[UniqueID]*datapb.SegmentInfo{ map[util.UniqueID]*datapb.SegmentInfo{
200: getSegmentInfo(200, 50000), 200: getSegmentInfo(200, 50000),
300: getSegmentInfo(300, 50000), 300: getSegmentInfo(300, 50000),
}, },
@ -377,7 +378,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
// for pChannel reuse on same collection // for pChannel reuse on same collection
{ {
"test insert msg with different channelName", "test insert msg with different channelName",
[]UniqueID{100}, []util.UniqueID{100},
nil, nil,
nil, nil,
getInsertMsgWithChannel(100, 10000, anotherChannelName), getInsertMsgWithChannel(100, 10000, anotherChannelName),
@ -405,10 +406,10 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
description string description string
segRemained bool segRemained bool
segTs Timestamp segTs util.Timestamp
msgTs Timestamp msgTs util.Timestamp
sealedSegInfo map[UniqueID]*datapb.SegmentInfo sealedSegInfo map[util.UniqueID]*datapb.SegmentInfo
inMsg *msgstream.InsertMsg inMsg *msgstream.InsertMsg
msgFiltered bool msgFiltered bool
}{ }{
@ -417,7 +418,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
true, true,
50000, 50000,
10000, 10000,
map[UniqueID]*datapb.SegmentInfo{ map[util.UniqueID]*datapb.SegmentInfo{
100: getSegmentInfo(100, 50000), 100: getSegmentInfo(100, 50000),
101: getSegmentInfo(101, 50000), 101: getSegmentInfo(101, 50000),
}, },
@ -429,7 +430,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
true, true,
50000, 50000,
10000, 10000,
map[UniqueID]*datapb.SegmentInfo{ map[util.UniqueID]*datapb.SegmentInfo{
100: getSegmentInfo(100, 50000), 100: getSegmentInfo(100, 50000),
101: getSegmentInfo(101, 50000), 101: getSegmentInfo(101, 50000),
}, },
@ -441,7 +442,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
false, false,
50000, 50000,
10000, 10000,
map[UniqueID]*datapb.SegmentInfo{ map[util.UniqueID]*datapb.SegmentInfo{
100: getSegmentInfo(100, 70000), 100: getSegmentInfo(100, 70000),
101: getSegmentInfo(101, 50000), 101: getSegmentInfo(101, 50000),
}, },
@ -474,14 +475,14 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
description string description string
segRemained bool segRemained bool
growingSegInfo map[UniqueID]*datapb.SegmentInfo growingSegInfo map[util.UniqueID]*datapb.SegmentInfo
inMsg *msgstream.InsertMsg inMsg *msgstream.InsertMsg
msgFiltered bool msgFiltered bool
}{ }{
{ {
"msgTs<segTs", "msgTs<segTs",
true, true,
map[UniqueID]*datapb.SegmentInfo{ map[util.UniqueID]*datapb.SegmentInfo{
100: getSegmentInfo(100, 50000), 100: getSegmentInfo(100, 50000),
101: getSegmentInfo(101, 50000), 101: getSegmentInfo(101, 50000),
}, },
@ -491,7 +492,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
{ {
"msgTs==segTs", "msgTs==segTs",
true, true,
map[UniqueID]*datapb.SegmentInfo{ map[util.UniqueID]*datapb.SegmentInfo{
100: getSegmentInfo(100, 50000), 100: getSegmentInfo(100, 50000),
101: getSegmentInfo(101, 50000), 101: getSegmentInfo(101, 50000),
}, },
@ -501,7 +502,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
{ {
"msgTs>segTs", "msgTs>segTs",
false, false,
map[UniqueID]*datapb.SegmentInfo{ map[util.UniqueID]*datapb.SegmentInfo{
100: getSegmentInfo(100, 50000), 100: getSegmentInfo(100, 50000),
101: getSegmentInfo(101, 50000), 101: getSegmentInfo(101, 50000),
}, },
@ -535,7 +536,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
func TestFlowGraph_DDNode_isDropped(t *testing.T) { func TestFlowGraph_DDNode_isDropped(t *testing.T) {
tests := []struct { tests := []struct {
indroppedSegment []*datapb.SegmentInfo indroppedSegment []*datapb.SegmentInfo
inSeg UniqueID inSeg util.UniqueID
expectedOut bool 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{ return &datapb.SegmentInfo{
ID: segmentID, ID: segmentID,
DmlPosition: &msgpb.MsgPosition{Timestamp: ts}, 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) 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{ return &msgstream.InsertMsg{
BaseMsg: msgstream.BaseMsg{EndTimestamp: ts}, BaseMsg: msgstream.BaseMsg{EndTimestamp: ts},
InsertRequest: msgpb.InsertRequest{ InsertRequest: msgpb.InsertRequest{

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package pipeline
import ( import (
"context" "context"
@ -66,8 +66,8 @@ func newDmInputNode(initCtx context.Context, dispatcherClient msgdispatcher.Clie
node := flowgraph.NewInputNode( node := flowgraph.NewInputNode(
input, input,
name, name,
Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32(), paramtable.Get().DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32(),
Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32(), paramtable.Get().DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32(),
typeutil.DataNodeRole, typeutil.DataNodeRole,
dmNodeConfig.serverID, dmNodeConfig.serverID,
dmNodeConfig.collectionID, dmNodeConfig.collectionID,

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package pipeline
import ( import (
"context" "context"

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package pipeline
import ( import (
"context" "context"
@ -22,8 +22,7 @@ import (
"go.uber.org/zap" "go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "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/log"
"github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/paramtable"
@ -31,14 +30,13 @@ import (
) )
type FlowgraphManager interface { type FlowgraphManager interface {
AddFlowgraph(ds *dataSyncService) AddFlowgraph(ds *DataSyncService)
AddandStartWithEtcdTickler(dn *DataNode, vchan *datapb.VchannelInfo, schema *schemapb.CollectionSchema, tickler *etcdTickler) error
RemoveFlowgraph(channel string) RemoveFlowgraph(channel string)
ClearFlowgraphs() ClearFlowgraphs()
GetFlowgraphService(channel string) (*dataSyncService, bool) GetFlowgraphService(channel string) (*DataSyncService, bool)
HasFlowgraph(channel string) bool HasFlowgraph(channel string) bool
HasFlowgraphWithOpID(channel string, opID UniqueID) bool HasFlowgraphWithOpID(channel string, opID int64) bool
GetFlowgraphCount() int GetFlowgraphCount() int
GetCollectionIDs() []int64 GetCollectionIDs() []int64
@ -50,58 +48,36 @@ var _ FlowgraphManager = (*fgManagerImpl)(nil)
type fgManagerImpl struct { type fgManagerImpl struct {
ctx context.Context ctx context.Context
cancelFunc context.CancelFunc 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()) ctx, cancelFunc := context.WithCancel(context.TODO())
return &fgManagerImpl{ return &fgManagerImpl{
ctx: ctx, ctx: ctx,
cancelFunc: cancelFunc, 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) fm.flowgraphs.Insert(ds.vchannelName, ds)
metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Inc() 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) { func (fm *fgManagerImpl) RemoveFlowgraph(channel string) {
if fg, loaded := fm.flowgraphs.Get(channel); loaded { if fg, loaded := fm.flowgraphs.Get(channel); loaded {
fg.close() fg.close()
fm.flowgraphs.Remove(channel) fm.flowgraphs.Remove(channel)
metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Dec() metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Dec()
rateCol.removeFlowGraphChannel(channel) util.RateCol.RemoveFlowGraphChannel(channel)
} }
} }
func (fm *fgManagerImpl) ClearFlowgraphs() { func (fm *fgManagerImpl) ClearFlowgraphs() {
log.Info("start drop all flowgraph resources in DataNode") 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() value.GracefullyClose()
fm.flowgraphs.GetAndRemove(key) 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) return fm.flowgraphs.Get(channel)
} }
@ -119,7 +95,7 @@ func (fm *fgManagerImpl) HasFlowgraph(channel string) bool {
return exist 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) ds, exist := fm.flowgraphs.Get(channel)
return exist && ds.opID == opID return exist && ds.opID == opID
} }
@ -131,7 +107,7 @@ func (fm *fgManagerImpl) GetFlowgraphCount() int {
func (fm *fgManagerImpl) GetCollectionIDs() []int64 { func (fm *fgManagerImpl) GetCollectionIDs() []int64 {
collectionSet := typeutil.UniqueSet{} 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()) collectionSet.Insert(value.metacache.Collection())
return true return true
}) })

View 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,
}
}

View File

@ -14,10 +14,11 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package pipeline
import ( import (
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "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/storage"
"github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/internal/util/flowgraph"
"github.com/milvus-io/milvus/pkg/mq/msgstream" "github.com/milvus-io/milvus/pkg/mq/msgstream"
@ -42,38 +43,24 @@ type (
Blob = storage.Blob Blob = storage.Blob
) )
type flowGraphMsg struct { type FlowGraphMsg struct {
BaseMsg BaseMsg
insertMessages []*msgstream.InsertMsg InsertMessages []*msgstream.InsertMsg
deleteMessages []*msgstream.DeleteMsg DeleteMessages []*msgstream.DeleteMsg
timeRange TimeRange TimeRange util.TimeRange
startPositions []*msgpb.MsgPosition StartPositions []*msgpb.MsgPosition
endPositions []*msgpb.MsgPosition EndPositions []*msgpb.MsgPosition
// segmentsToSync is the signal used by insertBufferNode to notify deleteNode to flush // segmentsToSync is the signal used by insertBufferNode to notify deleteNode to flush
segmentsToSync []UniqueID segmentsToSync []util.UniqueID
dropCollection bool dropCollection bool
dropPartitions []UniqueID dropPartitions []util.UniqueID
} }
func (fgMsg *flowGraphMsg) TimeTick() Timestamp { func (fgMsg *FlowGraphMsg) TimeTick() util.Timestamp {
return fgMsg.timeRange.timestampMax return fgMsg.TimeRange.TimestampMax
} }
func (fgMsg *flowGraphMsg) IsClose() bool { func (fgMsg *FlowGraphMsg) IsClose() bool {
return fgMsg.BaseMsg.IsCloseMsg() 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
}

View File

@ -14,17 +14,19 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package pipeline
import ( import (
"testing" "testing"
"github.com/stretchr/testify/assert" "github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus/internal/datanode/util"
) )
func TestInsertMsg_TimeTick(te *testing.T) { func TestInsertMsg_TimeTick(te *testing.T) {
tests := []struct { tests := []struct {
timeTimestanpMax Timestamp timeTimestanpMax util.Timestamp
description string description string
}{ }{
@ -34,7 +36,7 @@ func TestInsertMsg_TimeTick(te *testing.T) {
for _, test := range tests { for _, test := range tests {
te.Run(test.description, func(t *testing.T) { 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()) assert.Equal(t, test.timeTimestanpMax, fgMsg.TimeTick())
}) })
} }

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package pipeline
import ( import (
"sync/atomic" "sync/atomic"

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package pipeline
import ( import (
"fmt" "fmt"
@ -26,6 +26,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "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/metacache"
"github.com/milvus-io/milvus/internal/datanode/util"
"github.com/milvus-io/milvus/internal/datanode/writebuffer" "github.com/milvus-io/milvus/internal/datanode/writebuffer"
"github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/internal/util/flowgraph"
"github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/log"
@ -42,7 +43,7 @@ type ttNode struct {
metacache metacache.MetaCache metacache metacache.MetaCache
writeBufferManager writebuffer.BufferManager writeBufferManager writebuffer.BufferManager
lastUpdateTime *atomic.Time lastUpdateTime *atomic.Time
cpUpdater *channelCheckpointUpdater cpUpdater *util.ChannelCheckpointUpdater
dropMode *atomic.Bool dropMode *atomic.Bool
} }
@ -55,7 +56,7 @@ func (ttn *ttNode) IsValidInMsg(in []Msg) bool {
if !ttn.BaseNode.IsValidInMsg(in) { if !ttn.BaseNode.IsValidInMsg(in) {
return false return false
} }
_, ok := in[0].(*flowGraphMsg) _, ok := in[0].(*FlowGraphMsg)
if !ok { if !ok {
log.Warn("type assertion failed for flowGraphMsg", zap.String("name", reflect.TypeOf(in[0]).Name())) log.Warn("type assertion failed for flowGraphMsg", zap.String("name", reflect.TypeOf(in[0]).Name()))
return false return false
@ -68,7 +69,7 @@ func (ttn *ttNode) Close() {
// Operate handles input messages, implementing flowgraph.Node // Operate handles input messages, implementing flowgraph.Node
func (ttn *ttNode) Operate(in []Msg) []Msg { func (ttn *ttNode) Operate(in []Msg) []Msg {
fgMsg := in[0].(*flowGraphMsg) fgMsg := in[0].(*FlowGraphMsg)
if fgMsg.dropCollection { if fgMsg.dropCollection {
ttn.dropMode.Store(true) ttn.dropMode.Store(true)
} }
@ -80,9 +81,9 @@ func (ttn *ttNode) Operate(in []Msg) []Msg {
return []Msg{} return []Msg{}
} }
curTs, _ := tsoutil.ParseTS(fgMsg.timeRange.timestampMax) curTs, _ := tsoutil.ParseTS(fgMsg.TimeRange.TimestampMax)
if fgMsg.IsCloseMsg() { if fgMsg.IsCloseMsg() {
if len(fgMsg.endPositions) > 0 { if len(fgMsg.EndPositions) > 0 {
channelPos, _, err := ttn.writeBufferManager.GetCheckpoint(ttn.vChannelName) channelPos, _, err := ttn.writeBufferManager.GetCheckpoint(ttn.vChannelName)
if err != nil { if err != nil {
log.Warn("channel removed", zap.String("channel", ttn.vChannelName), zap.Error(err)) 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) 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 := BaseNode{}
baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32()) baseNode.SetMaxQueueLength(paramtable.Get().DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32())
baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32()) baseNode.SetMaxParallelism(paramtable.Get().DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32())
tt := &ttNode{ tt := &ttNode{
BaseNode: baseNode, BaseNode: baseNode,

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package pipeline
import ( import (
"sync" "sync"
@ -24,10 +24,12 @@ import (
"go.uber.org/zap" "go.uber.org/zap"
"golang.org/x/exp/maps" "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/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` // 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) // it makes sure time ticker send at most 10 times a second (1tick/100millisecond)
@ -46,11 +48,6 @@ type mergedTimeTickerSender struct {
closeOnce sync.Once closeOnce sync.Once
} }
var (
uniqueMergedTimeTickerSender *mergedTimeTickerSender
getUniqueMergedTimeTickerSender sync.Once
)
func newUniqueMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender { func newUniqueMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender {
return &mergedTimeTickerSender{ return &mergedTimeTickerSender{
ts: 0, // 0 for not tt send ts: 0, // 0 for not tt send
@ -61,17 +58,7 @@ func newUniqueMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender
} }
} }
func getOrCreateMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender { func (mt *mergedTimeTickerSender) bufferTs(ts util.Timestamp, segmentIDs []int64) {
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) {
mt.mu.Lock() mt.mu.Lock()
defer mt.mu.Unlock() defer mt.mu.Unlock()
mt.ts = ts mt.ts = ts
@ -83,7 +70,7 @@ func (mt *mergedTimeTickerSender) bufferTs(ts Timestamp, segmentIDs []int64) {
func (mt *mergedTimeTickerSender) tick() { func (mt *mergedTimeTickerSender) tick() {
defer mt.wg.Done() defer mt.wg.Done()
// this duration might be configuable in the future // 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() defer t.Stop()
for { for {
select { select {

View File

@ -1,4 +1,4 @@
package datanode package pipeline
import ( import (
"context" "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/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "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/metacache"
"github.com/milvus-io/milvus/internal/datanode/util"
"github.com/milvus-io/milvus/internal/datanode/writebuffer" "github.com/milvus-io/milvus/internal/datanode/writebuffer"
"github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/mq/msgstream" "github.com/milvus-io/milvus/pkg/mq/msgstream"
@ -27,7 +28,7 @@ type writeNode struct {
} }
func (wNode *writeNode) Operate(in []Msg) []Msg { func (wNode *writeNode) Operate(in []Msg) []Msg {
fgMsg := in[0].(*flowGraphMsg) fgMsg := in[0].(*FlowGraphMsg)
// close msg, ignore all data // close msg, ignore all data
if fgMsg.IsCloseMsg() { if fgMsg.IsCloseMsg() {
@ -35,31 +36,31 @@ func (wNode *writeNode) Operate(in []Msg) []Msg {
} }
// replace pchannel with vchannel // replace pchannel with vchannel
startPositions := make([]*msgpb.MsgPosition, 0, len(fgMsg.startPositions)) startPositions := make([]*msgpb.MsgPosition, 0, len(fgMsg.StartPositions))
for idx := range fgMsg.startPositions { for idx := range fgMsg.StartPositions {
pos := proto.Clone(fgMsg.startPositions[idx]).(*msgpb.MsgPosition) pos := proto.Clone(fgMsg.StartPositions[idx]).(*msgpb.MsgPosition)
pos.ChannelName = wNode.channelName pos.ChannelName = wNode.channelName
startPositions = append(startPositions, pos) startPositions = append(startPositions, pos)
} }
fgMsg.startPositions = startPositions fgMsg.StartPositions = startPositions
endPositions := make([]*msgpb.MsgPosition, 0, len(fgMsg.endPositions)) endPositions := make([]*msgpb.MsgPosition, 0, len(fgMsg.EndPositions))
for idx := range fgMsg.endPositions { for idx := range fgMsg.EndPositions {
pos := proto.Clone(fgMsg.endPositions[idx]).(*msgpb.MsgPosition) pos := proto.Clone(fgMsg.EndPositions[idx]).(*msgpb.MsgPosition)
pos.ChannelName = wNode.channelName pos.ChannelName = wNode.channelName
endPositions = append(endPositions, pos) endPositions = append(endPositions, pos)
} }
fgMsg.endPositions = endPositions fgMsg.EndPositions = endPositions
if len(fgMsg.startPositions) == 0 { if len(fgMsg.StartPositions) == 0 {
return []Msg{} return []Msg{}
} }
if len(fgMsg.endPositions) == 0 { if len(fgMsg.EndPositions) == 0 {
return []Msg{} return []Msg{}
} }
var spans []trace.Span var spans []trace.Span
for _, msg := range fgMsg.insertMessages { for _, msg := range fgMsg.InsertMessages {
ctx, sp := startTracer(msg, "WriteNode") ctx, sp := util.StartTracer(msg, "WriteNode")
spans = append(spans, sp) spans = append(spans, sp)
msg.SetTraceCtx(ctx) 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 { if err != nil {
log.Error("failed to buffer data", zap.Error(err)) log.Error("failed to buffer data", zap.Error(err))
panic(err) panic(err)
} }
stats := lo.FilterMap( 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) { func(id int64, _ int) (*commonpb.SegmentStats, bool) {
segInfo, ok := wNode.metacache.GetSegmentByID(id) segInfo, ok := wNode.metacache.GetSegmentByID(id)
if !ok { if !ok {
@ -91,12 +92,12 @@ func (wNode *writeNode) Operate(in []Msg) []Msg {
}, true }, true
}) })
wNode.updater.update(wNode.channelName, end.GetTimestamp(), stats) wNode.updater.Update(wNode.channelName, end.GetTimestamp(), stats)
res := flowGraphMsg{ res := FlowGraphMsg{
timeRange: fgMsg.timeRange, TimeRange: fgMsg.TimeRange,
startPositions: fgMsg.startPositions, StartPositions: fgMsg.StartPositions,
endPositions: fgMsg.endPositions, EndPositions: fgMsg.EndPositions,
dropCollection: fgMsg.dropCollection, dropCollection: fgMsg.dropCollection,
} }

View File

@ -1,13 +1,8 @@
// Code generated by mockery v2.32.4. DO NOT EDIT. // Code generated by mockery v2.32.4. DO NOT EDIT.
package datanode package pipeline
import ( import mock "github.com/stretchr/testify/mock"
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"
)
// MockFlowgraphManager is an autogenerated mock type for the FlowgraphManager type // MockFlowgraphManager is an autogenerated mock type for the FlowgraphManager type
type MockFlowgraphManager struct { type MockFlowgraphManager struct {
@ -23,7 +18,7 @@ func (_m *MockFlowgraphManager) EXPECT() *MockFlowgraphManager_Expecter {
} }
// AddFlowgraph provides a mock function with given fields: ds // AddFlowgraph provides a mock function with given fields: ds
func (_m *MockFlowgraphManager) AddFlowgraph(ds *dataSyncService) { func (_m *MockFlowgraphManager) AddFlowgraph(ds *DataSyncService) {
_m.Called(ds) _m.Called(ds)
} }
@ -33,14 +28,14 @@ type MockFlowgraphManager_AddFlowgraph_Call struct {
} }
// AddFlowgraph is a helper method to define mock.On call // AddFlowgraph is a helper method to define mock.On call
// - ds *dataSyncService // - ds *DataSyncService
func (_e *MockFlowgraphManager_Expecter) AddFlowgraph(ds interface{}) *MockFlowgraphManager_AddFlowgraph_Call { func (_e *MockFlowgraphManager_Expecter) AddFlowgraph(ds interface{}) *MockFlowgraphManager_AddFlowgraph_Call {
return &MockFlowgraphManager_AddFlowgraph_Call{Call: _e.mock.On("AddFlowgraph", ds)} 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) { _c.Call.Run(func(args mock.Arguments) {
run(args[0].(*dataSyncService)) run(args[0].(*DataSyncService))
}) })
return _c return _c
} }
@ -50,52 +45,7 @@ func (_c *MockFlowgraphManager_AddFlowgraph_Call) Return() *MockFlowgraphManager
return _c return _c
} }
func (_c *MockFlowgraphManager_AddFlowgraph_Call) RunAndReturn(run func(*dataSyncService)) *MockFlowgraphManager_AddFlowgraph_Call { 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 {
_c.Call.Return(run) _c.Call.Return(run)
return _c return _c
} }
@ -249,19 +199,19 @@ func (_c *MockFlowgraphManager_GetFlowgraphCount_Call) RunAndReturn(run func() i
} }
// GetFlowgraphService provides a mock function with given fields: channel // 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) ret := _m.Called(channel)
var r0 *dataSyncService var r0 *DataSyncService
var r1 bool 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) 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) r0 = rf(channel)
} else { } else {
if ret.Get(0) != nil { 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 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) _c.Call.Return(_a0, _a1)
return _c 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) _c.Call.Return(run)
return _c return _c
} }

View File

@ -1,4 +1,4 @@
package datanode package pipeline
import ( import (
"fmt" "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/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "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/metrics"
"github.com/milvus-io/milvus/pkg/mq/msgstream" "github.com/milvus-io/milvus/pkg/mq/msgstream"
"github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/commonpbutil"
@ -16,7 +17,7 @@ import (
) )
type statsUpdater interface { 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 // mqStatsUpdater is the wrapper of mergedTimeTickSender
@ -40,7 +41,7 @@ func newMqStatsUpdater(config *nodeConfig, producer msgstream.MsgStream) statsUp
return updater return updater
} }
func (u *mqStatsUpdater) send(ts Timestamp, segmentIDs []int64) error { func (u *mqStatsUpdater) send(ts util.Timestamp, segmentIDs []int64) error {
u.mut.Lock() u.mut.Lock()
defer u.mut.Unlock() defer u.mut.Unlock()
stats := lo.Map(segmentIDs, func(id int64, _ int) *commonpb.SegmentStats { 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 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() u.mut.Lock()
defer u.mut.Unlock() defer u.mut.Unlock()
segmentIDs := lo.Map(stats, func(stats *commonpb.SegmentStats, _ int) int64 { return stats.SegmentID }) segmentIDs := lo.Map(stats, func(stats *commonpb.SegmentStats, _ int) int64 { return stats.SegmentID })

View File

@ -1,4 +1,4 @@
package datanode package pipeline
import ( import (
"testing" "testing"

View File

@ -32,7 +32,6 @@ import (
"github.com/milvus-io/milvus/internal/datanode/importv2" "github.com/milvus-io/milvus/internal/datanode/importv2"
"github.com/milvus-io/milvus/internal/datanode/io" "github.com/milvus-io/milvus/internal/datanode/io"
"github.com/milvus-io/milvus/internal/datanode/metacache" "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/metastore/kv/binlog"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "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{}{} allSegments[segID] = struct{}{}
} }
missingSegments := ds.metacache.DetectMissingSegments(allSegments) missingSegments := ds.GetMetaCache().DetectMissingSegments(allSegments)
newSegments := make([]*datapb.SyncSegmentInfo, 0, len(missingSegments)) newSegments := make([]*datapb.SyncSegmentInfo, 0, len(missingSegments))
futures := make([]*conc.Future[any], 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)) log.Warn("failed to DecompressBinLog", zap.Error(err))
return val, 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 { if err != nil {
log.Warn("failed to load segment stats log", zap.Error(err)) log.Warn("failed to load segment stats log", zap.Error(err))
return val, err return val, err
@ -343,7 +342,7 @@ func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegments
return future.Value().(*metacache.BloomFilterSet) 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 return merr.Success(), nil
} }

View File

@ -36,6 +36,8 @@ import (
"github.com/milvus-io/milvus/internal/datanode/broker" "github.com/milvus-io/milvus/internal/datanode/broker"
"github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/datanode/compaction"
"github.com/milvus-io/milvus/internal/datanode/metacache" "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/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/storage"
@ -77,7 +79,7 @@ func (s *DataNodeServicesSuite) SetupSuite() {
} }
func (s *DataNodeServicesSuite) SetupTest() { 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) s.node.SetEtcdClient(s.etcdCli)
err := s.node.Init() err := s.node.Init()
@ -292,18 +294,25 @@ func (s *DataNodeServicesSuite) TestFlushSegments() {
FlushedSegmentIds: []int64{}, FlushedSegmentIds: []int64{},
} }
err := s.node.flowgraphManager.AddandStartWithEtcdTickler(s.node, vchan, schema, genTestTickler()) chanWathInfo := &datapb.ChannelWatchInfo{
s.Require().NoError(err) Vchan: vchan,
State: datapb.ChannelWatchState_WatchSuccess,
fgservice, ok := s.node.flowgraphManager.GetFlowgraphService(dmChannelName) Schema: schema,
s.Require().True(ok) }
metaCache := metacache.NewMockMetaCache(s.T()) metaCache := metacache.NewMockMetaCache(s.T())
metaCache.EXPECT().Collection().Return(1).Maybe() metaCache.EXPECT().Collection().Return(1).Maybe()
metaCache.EXPECT().Schema().Return(schema).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, ID: segmentID,
CollectionID: 1, CollectionID: 1,
PartitionID: 2, PartitionID: 2,
@ -419,7 +428,7 @@ func (s *DataNodeServicesSuite) TestShowConfigurations() {
func (s *DataNodeServicesSuite) TestGetMetrics() { func (s *DataNodeServicesSuite) TestGetMetrics() {
node := &DataNode{} node := &DataNode{}
node.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}}) node.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}})
node.flowgraphManager = newFlowgraphManager() node.flowgraphManager = pipeline.NewFlowgraphManager()
// server is closed // server is closed
node.stateCode.Store(commonpb.StateCode_Abnormal) node.stateCode.Store(commonpb.StateCode_Abnormal)
resp, err := node.GetMetrics(s.ctx, &milvuspb.GetMetricsRequest{}) resp, err := node.GetMetrics(s.ctx, &milvuspb.GetMetricsRequest{})
@ -618,10 +627,9 @@ func (s *DataNodeServicesSuite) TestSyncSegments() {
}, func(*datapb.SegmentInfo) *metacache.BloomFilterSet { }, func(*datapb.SegmentInfo) *metacache.BloomFilterSet {
return metacache.NewBloomFilterSet() return metacache.NewBloomFilterSet()
}) })
mockFlowgraphManager := NewMockFlowgraphManager(s.T()) mockFlowgraphManager := pipeline.NewMockFlowgraphManager(s.T())
mockFlowgraphManager.EXPECT().GetFlowgraphService(mock.Anything).Return(&dataSyncService{ mockFlowgraphManager.EXPECT().GetFlowgraphService(mock.Anything).
metacache: cache, Return(pipeline.NewDataSyncServiceWithMetaCache(cache), true)
}, true)
s.node.flowgraphManager = mockFlowgraphManager s.node.flowgraphManager = mockFlowgraphManager
ctx := context.Background() ctx := context.Background()
req := &datapb.SyncSegmentsRequest{ req := &datapb.SyncSegmentsRequest{

View File

@ -117,7 +117,7 @@ type MockTask_HandleError_Call struct {
} }
// HandleError is a helper method to define mock.On call // HandleError is a helper method to define mock.On call
// - _a0 error // - _a0 error
func (_e *MockTask_Expecter) HandleError(_a0 interface{}) *MockTask_HandleError_Call { func (_e *MockTask_Expecter) HandleError(_a0 interface{}) *MockTask_HandleError_Call {
return &MockTask_HandleError_Call{Call: _e.mock.On("HandleError", _a0)} return &MockTask_HandleError_Call{Call: _e.mock.On("HandleError", _a0)}
} }

View File

@ -14,9 +14,11 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // 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. // Cache stores flushing segments' ids to prevent flushing the same segment again and again.
// //
@ -29,8 +31,8 @@ type Cache struct {
*typeutil.ConcurrentSet[UniqueID] *typeutil.ConcurrentSet[UniqueID]
} }
// newCache returns a new Cache // NewCache returns a new Cache
func newCache() *Cache { func NewCache() *Cache {
return &Cache{ return &Cache{
ConcurrentSet: typeutil.NewConcurrentSet[UniqueID](), ConcurrentSet: typeutil.NewConcurrentSet[UniqueID](),
} }

View File

@ -14,16 +14,29 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package util
import ( import (
"os"
"testing" "testing"
"github.com/stretchr/testify/assert" "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) { func TestSegmentCache(t *testing.T) {
segCache := newCache() segCache := NewCache()
assert.False(t, segCache.checkIfCached(0)) assert.False(t, segCache.checkIfCached(0))

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package util
import ( import (
"context" "context"
@ -41,7 +41,7 @@ type channelCPUpdateTask struct {
flush bool // indicates whether the task originates from flush flush bool // indicates whether the task originates from flush
} }
type channelCheckpointUpdater struct { type ChannelCheckpointUpdater struct {
broker broker.Broker broker broker.Broker
mu sync.RWMutex mu sync.RWMutex
@ -52,8 +52,8 @@ type channelCheckpointUpdater struct {
closeOnce sync.Once closeOnce sync.Once
} }
func newChannelCheckpointUpdater(broker broker.Broker) *channelCheckpointUpdater { func NewChannelCheckpointUpdater(broker broker.Broker) *ChannelCheckpointUpdater {
return &channelCheckpointUpdater{ return &ChannelCheckpointUpdater{
broker: broker, broker: broker,
tasks: make(map[string]*channelCPUpdateTask), tasks: make(map[string]*channelCPUpdateTask),
closeCh: make(chan struct{}), 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") log.Info("channel checkpoint updater start")
ticker := time.NewTicker(paramtable.Get().DataNodeCfg.ChannelCheckpointUpdateTickInSeconds.GetAsDuration(time.Second)) ticker := time.NewTicker(paramtable.Get().DataNodeCfg.ChannelCheckpointUpdateTickInSeconds.GetAsDuration(time.Second))
defer ticker.Stop() 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() ccu.mu.RLock()
defer ccu.mu.RUnlock() defer ccu.mu.RUnlock()
task, ok := ccu.tasks[channel] task, ok := ccu.tasks[channel]
return task, ok return task, ok
} }
func (ccu *channelCheckpointUpdater) trigger() { func (ccu *ChannelCheckpointUpdater) trigger() {
select { select {
case ccu.notifyChan <- struct{}{}: case ccu.notifyChan <- struct{}{}:
default: default:
} }
} }
func (ccu *channelCheckpointUpdater) updateCheckpoints(tasks []*channelCPUpdateTask) { func (ccu *ChannelCheckpointUpdater) updateCheckpoints(tasks []*channelCPUpdateTask) {
taskGroups := lo.Chunk(tasks, paramtable.Get().DataNodeCfg.MaxChannelCheckpointsPerRPC.GetAsInt()) taskGroups := lo.Chunk(tasks, paramtable.Get().DataNodeCfg.MaxChannelCheckpointsPerRPC.GetAsInt())
updateChanCPMaxParallel := paramtable.Get().DataNodeCfg.UpdateChannelCheckpointMaxParallel.GetAsInt() updateChanCPMaxParallel := paramtable.Get().DataNodeCfg.UpdateChannelCheckpointMaxParallel.GetAsInt()
if updateChanCPMaxParallel <= 0 { if updateChanCPMaxParallel <= 0 {
@ -152,7 +152,7 @@ func (ccu *channelCheckpointUpdater) updateCheckpoints(tasks []*channelCPUpdateT
}) })
} }
func (ccu *channelCheckpointUpdater) execute() { func (ccu *ChannelCheckpointUpdater) execute() {
ccu.mu.RLock() ccu.mu.RLock()
tasks := lo.Values(ccu.tasks) tasks := lo.Values(ccu.tasks)
ccu.mu.RUnlock() ccu.mu.RUnlock()
@ -160,7 +160,7 @@ func (ccu *channelCheckpointUpdater) execute() {
ccu.updateCheckpoints(tasks) 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() == "" { if channelPos == nil || channelPos.GetMsgID() == nil || channelPos.GetChannelName() == "" {
log.Warn("illegal checkpoint", zap.Any("pos", channelPos)) log.Warn("illegal checkpoint", zap.Any("pos", channelPos))
return 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() ccu.mu.RLock()
defer ccu.mu.RUnlock() defer ccu.mu.RUnlock()
return len(ccu.tasks) return len(ccu.tasks)
} }
func (ccu *channelCheckpointUpdater) close() { func (ccu *ChannelCheckpointUpdater) Close() {
ccu.closeOnce.Do(func() { ccu.closeOnce.Do(func() {
close(ccu.closeCh) close(ccu.closeCh)
}) })

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package util
import ( import (
"context" "context"
@ -36,12 +36,12 @@ type ChannelCPUpdaterSuite struct {
suite.Suite suite.Suite
broker *broker.MockBroker broker *broker.MockBroker
updater *channelCheckpointUpdater updater *ChannelCheckpointUpdater
} }
func (s *ChannelCPUpdaterSuite) SetupTest() { func (s *ChannelCPUpdaterSuite) SetupTest() {
s.broker = broker.NewMockBroker(s.T()) s.broker = broker.NewMockBroker(s.T())
s.updater = newChannelCheckpointUpdater(s.broker) s.updater = NewChannelCheckpointUpdater(s.broker)
} }
func (s *ChannelCPUpdaterSuite) TestUpdate() { func (s *ChannelCPUpdaterSuite) TestUpdate() {
@ -53,8 +53,8 @@ func (s *ChannelCPUpdaterSuite) TestUpdate() {
return nil return nil
}) })
go s.updater.start() go s.updater.Start()
defer s.updater.close() defer s.updater.Close()
tasksNum := 100000 tasksNum := 100000
counter := atomic.NewInt64(0) counter := atomic.NewInt64(0)

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package util
import ( import (
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
@ -22,8 +22,8 @@ import (
"github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/proto/etcdpb"
) )
// reviseVChannelInfo will revise the datapb.VchannelInfo for upgrade compatibility from 2.0.2 // ReviseVChannelInfo will revise the datapb.VchannelInfo for upgrade compatibility from 2.0.2
func reviseVChannelInfo(vChannel *datapb.VchannelInfo) { func ReviseVChannelInfo(vChannel *datapb.VchannelInfo) {
removeDuplicateSegmentIDFn := func(ids []int64) []int64 { removeDuplicateSegmentIDFn := func(ids []int64) []int64 {
result := make([]int64, 0, len(ids)) result := make([]int64, 0, len(ids))
existDict := make(map[int64]bool) existDict := make(map[int64]bool)

View File

@ -14,65 +14,76 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package util
import ( import (
"sync" "sync"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/ratelimitutil" "github.com/milvus-io/milvus/pkg/util/ratelimitutil"
"github.com/milvus-io/milvus/pkg/util/typeutil" "github.com/milvus-io/milvus/pkg/util/typeutil"
) )
// rateCol is global rateCollector in DataNode. // RateCol is global RateCollector in DataNode.
var ( var (
rateCol *rateCollector RateCol *RateCollector
initOnce sync.Once initOnce sync.Once
) )
// rateCollector helps to collect and calculate values (like rate, timeTick and etc...). // RateCollector helps to collect and calculate values (like rate, timeTick and etc...).
type rateCollector struct { type RateCollector struct {
*ratelimitutil.RateCollector *ratelimitutil.RateCollector
flowGraphTtMu sync.Mutex flowGraphTtMu sync.Mutex
flowGraphTt map[string]Timestamp flowGraphTt map[string]Timestamp
} }
func initGlobalRateCollector() error { func InitGlobalRateCollector() error {
var err error var err error
initOnce.Do(func() { initOnce.Do(func() {
rateCol, err = newRateCollector() RateCol, err = NewRateCollector()
}) })
RateCol.Register(metricsinfo.InsertConsumeThroughput)
RateCol.Register(metricsinfo.DeleteConsumeThroughput)
return err return err
} }
// newRateCollector returns a new rateCollector. func DeregisterRateCollector(label string) {
func newRateCollector() (*rateCollector, error) { 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) rc, err := ratelimitutil.NewRateCollector(ratelimitutil.DefaultWindow, ratelimitutil.DefaultGranularity, false)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return &rateCollector{ return &RateCollector{
RateCollector: rc, RateCollector: rc,
flowGraphTt: make(map[string]Timestamp), flowGraphTt: make(map[string]Timestamp),
}, nil }, nil
} }
// updateFlowGraphTt updates rateCollector's flow graph time tick. // UpdateFlowGraphTt updates RateCollector's flow graph time tick.
func (r *rateCollector) updateFlowGraphTt(channel string, t Timestamp) { func (r *RateCollector) UpdateFlowGraphTt(channel string, t Timestamp) {
r.flowGraphTtMu.Lock() r.flowGraphTtMu.Lock()
defer r.flowGraphTtMu.Unlock() defer r.flowGraphTtMu.Unlock()
r.flowGraphTt[channel] = t r.flowGraphTt[channel] = t
} }
// removeFlowGraphChannel removes channel from flowGraphTt. // RemoveFlowGraphChannel removes channel from flowGraphTt.
func (r *rateCollector) removeFlowGraphChannel(channel string) { func (r *RateCollector) RemoveFlowGraphChannel(channel string) {
r.flowGraphTtMu.Lock() r.flowGraphTtMu.Lock()
defer r.flowGraphTtMu.Unlock() defer r.flowGraphTtMu.Unlock()
delete(r.flowGraphTt, channel) delete(r.flowGraphTt, channel)
} }
// getMinFlowGraphTt returns the vchannel and minimal time tick of flow graphs. // GetMinFlowGraphTt returns the vchannel and minimal time tick of flow graphs.
func (r *rateCollector) getMinFlowGraphTt() (string, Timestamp) { func (r *RateCollector) GetMinFlowGraphTt() (string, Timestamp) {
r.flowGraphTtMu.Lock() r.flowGraphTtMu.Lock()
defer r.flowGraphTtMu.Unlock() defer r.flowGraphTtMu.Unlock()
minTt := typeutil.MaxTimestamp minTt := typeutil.MaxTimestamp

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package util
import ( import (
"testing" "testing"
@ -26,16 +26,16 @@ import (
func TestRateCollector(t *testing.T) { func TestRateCollector(t *testing.T) {
t.Run("test FlowGraphTt", func(t *testing.T) { t.Run("test FlowGraphTt", func(t *testing.T) {
collector, err := newRateCollector() collector, err := NewRateCollector()
assert.NoError(t, err) assert.NoError(t, err)
c, minTt := collector.getMinFlowGraphTt() c, minTt := collector.GetMinFlowGraphTt()
assert.Equal(t, "", c) assert.Equal(t, "", c)
assert.Equal(t, typeutil.MaxTimestamp, minTt) assert.Equal(t, typeutil.MaxTimestamp, minTt)
collector.updateFlowGraphTt("channel1", 100) collector.UpdateFlowGraphTt("channel1", 100)
collector.updateFlowGraphTt("channel2", 200) collector.UpdateFlowGraphTt("channel2", 200)
collector.updateFlowGraphTt("channel3", 50) collector.UpdateFlowGraphTt("channel3", 50)
c, minTt = collector.getMinFlowGraphTt() c, minTt = collector.GetMinFlowGraphTt()
assert.Equal(t, "channel3", c) assert.Equal(t, "channel3", c)
assert.Equal(t, Timestamp(50), minTt) assert.Equal(t, Timestamp(50), minTt)
}) })

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package util
import ( import (
"bytes" "bytes"
@ -22,12 +22,8 @@ import (
"encoding/binary" "encoding/binary"
"fmt" "fmt"
"math" "math"
"math/rand"
"time"
"github.com/cockroachdb/errors" "github.com/cockroachdb/errors"
"github.com/samber/lo"
"github.com/stretchr/testify/mock"
"go.uber.org/zap" "go.uber.org/zap"
"google.golang.org/grpc" "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/milvuspb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "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-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/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/datapb"
"github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/proto/rootcoordpb"
"github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/dependency" "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/common"
"github.com/milvus-io/milvus/pkg/kv"
"github.com/milvus-io/milvus/pkg/log" "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/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/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 // As used in data_sync_service_test.go
var segID2SegInfo = map[int64]*datapb.SegmentInfo{ 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{} type MetaFactory struct{}
func NewMetaFactory() *MetaFactory { func NewMetaFactory() *MetaFactory {
@ -877,66 +791,6 @@ func (df *DataFactory) GenMsgStreamDeleteMsgWithTs(idx int, pks []storage.Primar
return msg 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) { func (m *RootCoordFactory) setCollectionID(id UniqueID) {
m.collectionID = id m.collectionID = id
} }
@ -1065,139 +919,12 @@ func (f *FailMessageStreamFactory) NewTtMsgStream(ctx context.Context) (msgstrea
return nil, errors.New("mocked failure") 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 // MockDataSuiteBase compose some mock dependency to generate test dataset
type MockDataSuiteBase struct { type MockDataSuiteBase struct {
schema *schemapb.CollectionSchema schema *schemapb.CollectionSchema
} }
func (s *MockDataSuiteBase) prepareData() { func (s *MockDataSuiteBase) PrepareData() {
s.schema = &schemapb.CollectionSchema{ s.schema = &schemapb.CollectionSchema{
Name: "test_collection", Name: "test_collection",
Fields: []*schemapb.FieldSchema{ Fields: []*schemapb.FieldSchema{
@ -1214,3 +941,34 @@ func (s *MockDataSuiteBase) prepareData() {
func EmptyBfsFactory(info *datapb.SegmentInfo) *metacache.BloomFilterSet { func EmptyBfsFactory(info *datapb.SegmentInfo) *metacache.BloomFilterSet {
return metacache.NewBloomFilterSet() 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"},
},
},
},
},
}
}

View 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),
}
}

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package util
import ( import (
"context" "context"
@ -29,13 +29,14 @@ import (
"github.com/milvus-io/milvus/internal/datanode/broker" "github.com/milvus-io/milvus/internal/datanode/broker"
"github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/commonpbutil"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/retry" "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 is to merge channel states updated by flow graph node and send to datacoord periodically
// timeTickSender hold segmentStats cache for each channel, // TimeTickSender hold segmentStats cache for each channel,
// after send succeeds will clean the cache earlier than last sent timestamp // after send succeeds will clean the cache earlier than last sent timestamp
type timeTickSender struct { type TimeTickSender struct {
nodeID int64 nodeID int64
broker broker.Broker broker broker.Broker
@ -53,22 +54,23 @@ type channelStats struct {
lastTs uint64 lastTs uint64
} }
// data struct only used in timeTickSender // data struct only used in TimeTickSender
type segmentStats struct { type segmentStats struct {
*commonpb.SegmentStats *commonpb.SegmentStats
ts uint64 ts uint64
} }
func newTimeTickSender(broker broker.Broker, nodeID int64, opts ...retry.Option) *timeTickSender { func NewTimeTickSender(broker broker.Broker, nodeID int64, opts ...retry.Option) *TimeTickSender {
return &timeTickSender{ return &TimeTickSender{
nodeID: nodeID, nodeID: nodeID,
broker: broker, broker: broker,
statsCache: make(map[string]*channelStats), statsCache: make(map[string]*channelStats),
options: opts, options: opts,
mu: sync.RWMutex{},
} }
} }
func (m *timeTickSender) start() { func (m *TimeTickSender) Start() {
m.wg.Add(1) m.wg.Add(1)
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
m.cancelFunc = cancel m.cancelFunc = cancel
@ -76,22 +78,23 @@ func (m *timeTickSender) start() {
defer m.wg.Done() defer m.wg.Done()
m.work(ctx) m.work(ctx)
}() }()
log.Info("timeTick sender started")
} }
func (m *timeTickSender) Stop() { func (m *TimeTickSender) Stop() {
if m.cancelFunc != nil { if m.cancelFunc != nil {
m.cancelFunc() m.cancelFunc()
m.wg.Wait() m.wg.Wait()
} }
} }
func (m *timeTickSender) work(ctx context.Context) { func (m *TimeTickSender) work(ctx context.Context) {
ticker := time.NewTicker(Params.DataNodeCfg.DataNodeTimeTickInterval.GetAsDuration(time.Millisecond)) ticker := time.NewTicker(paramtable.Get().DataNodeCfg.DataNodeTimeTickInterval.GetAsDuration(time.Millisecond))
defer ticker.Stop() defer ticker.Stop()
for { for {
select { select {
case <-ctx.Done(): case <-ctx.Done():
log.Info("timeTickSender context done") log.Info("TimeTickSender context done")
return return
case <-ticker.C: case <-ticker.C:
m.sendReport(ctx) 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() m.mu.Lock()
defer m.mu.Unlock() defer m.mu.Unlock()
_, ok := m.statsCache[channelName] _, ok := m.statsCache[channelName]
@ -118,7 +121,7 @@ func (m *timeTickSender) update(channelName string, timestamp uint64, segStats [
m.statsCache[channelName].lastTs = timestamp 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() m.mu.RLock()
defer m.mu.RUnlock() defer m.mu.RUnlock()
@ -144,7 +147,7 @@ func (m *timeTickSender) assembleDatanodeTtMsg() ([]*msgpb.DataNodeTtMsg, map[st
return msgs, lastSentTss return msgs, lastSentTss
} }
func (m *timeTickSender) cleanStatesCache(lastSentTss map[string]uint64) { func (m *TimeTickSender) cleanStatesCache(lastSentTss map[string]uint64) {
m.mu.Lock() m.mu.Lock()
defer m.mu.Unlock() defer m.mu.Unlock()
sizeBeforeClean := len(m.statsCache) sizeBeforeClean := len(m.statsCache)
@ -161,12 +164,12 @@ func (m *timeTickSender) cleanStatesCache(lastSentTss map[string]uint64) {
delete(m.statsCache, channelName) 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() 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 { err := retry.Do(ctx, func() error {
return m.broker.ReportTimeTick(ctx, toSendMsgs) return m.broker.ReportTimeTick(ctx, toSendMsgs)
}, m.options...) }, m.options...)

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package util
import ( import (
"context" "context"
@ -40,7 +40,7 @@ func TestTimetickManagerNormal(t *testing.T) {
broker := broker.NewMockBroker(t) broker := broker.NewMockBroker(t)
broker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(nil).Maybe() broker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(nil).Maybe()
manager := newTimeTickSender(broker, 0) manager := NewTimeTickSender(broker, 0)
channelName1 := "channel1" channelName1 := "channel1"
ts := uint64(time.Now().UnixMilli()) ts := uint64(time.Now().UnixMilli())
@ -53,7 +53,7 @@ func TestTimetickManagerNormal(t *testing.T) {
}, },
} }
// update first time // update first time
manager.update(channelName1, ts, segmentStats) manager.Update(channelName1, ts, segmentStats)
chanStats, exist := manager.statsCache[channelName1] chanStats, exist := manager.statsCache[channelName1]
assert.Equal(t, true, exist) assert.Equal(t, true, exist)
@ -76,7 +76,7 @@ func TestTimetickManagerNormal(t *testing.T) {
}, },
} }
ts2 := ts + 100 ts2 := ts + 100
manager.update(channelName1, ts2, segmentStats2) manager.Update(channelName1, ts2, segmentStats2)
chanStats, exist = manager.statsCache[channelName1] chanStats, exist = manager.statsCache[channelName1]
assert.Equal(t, true, exist) assert.Equal(t, true, exist)
@ -106,7 +106,7 @@ func TestTimetickManagerNormal(t *testing.T) {
NumRows: 3333300, NumRows: 3333300,
}, },
} }
manager.update(channelName2, ts3, segmentStats3) manager.Update(channelName2, ts3, segmentStats3)
err := manager.sendReport(ctx) err := manager.sendReport(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -130,7 +130,7 @@ func TestTimetickManagerNormal(t *testing.T) {
NumRows: 3333300, NumRows: 3333300,
}, },
} }
manager.update(channelName3, ts4, segmentStats4) manager.Update(channelName3, ts4, segmentStats4)
err = manager.sendReport(ctx) err = manager.sendReport(ctx)
assert.NoError(t, err) assert.NoError(t, err)
@ -145,7 +145,7 @@ func TestTimetickManagerSendErr(t *testing.T) {
broker := broker.NewMockBroker(t) broker := broker.NewMockBroker(t)
broker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(errors.New("mock")).Maybe() 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" channelName1 := "channel1"
ts := uint64(time.Now().Unix()) ts := uint64(time.Now().Unix())
@ -157,7 +157,7 @@ func TestTimetickManagerSendErr(t *testing.T) {
}, },
} }
// update first time // update first time
manager.update(channelName1, ts, segmentStats) manager.Update(channelName1, ts, segmentStats)
err := manager.sendReport(ctx) err := manager.sendReport(ctx)
assert.Error(t, err) assert.Error(t, err)
} }
@ -174,8 +174,8 @@ func TestTimetickManagerSendReport(t *testing.T) {
}). }).
Return(nil) Return(nil)
mockDataCoord.EXPECT().ReportDataNodeTtMsgs(mock.Anything, mock.Anything).Return(merr.Status(nil), nil).Maybe() mockDataCoord.EXPECT().ReportDataNodeTtMsgs(mock.Anything, mock.Anything).Return(merr.Status(nil), nil).Maybe()
manager := newTimeTickSender(broker, 0) manager := NewTimeTickSender(broker, 0)
manager.start() manager.Start()
assert.Eventually(t, func() bool { assert.Eventually(t, func() bool {
return called.Load() return called.Load()

View File

@ -14,7 +14,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package datanode package util
import ( import (
"context" "context"
@ -22,6 +22,15 @@ import (
"go.opentelemetry.io/otel" "go.opentelemetry.io/otel"
"go.opentelemetry.io/otel/trace" "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/mq/msgstream"
"github.com/milvus-io/milvus/pkg/util/typeutil" "github.com/milvus-io/milvus/pkg/util/typeutil"
) )
@ -40,13 +49,28 @@ type (
DSL = string DSL = string
) )
// TimeRange is a range of timestamp contains the min-timestamp and max-timestamp type PipelineParams struct {
type TimeRange struct { Ctx context.Context
timestampMin Timestamp Broker broker.Broker
timestampMax Timestamp 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() ctx := msg.TraceCtx()
if ctx == nil { if ctx == nil {
ctx = context.Background() ctx = context.Background()

View File

@ -2833,7 +2833,6 @@ user-task-polling:
type dataCoordConfig struct { type dataCoordConfig struct {
// --- CHANNEL --- // --- CHANNEL ---
WatchTimeoutInterval ParamItem `refreshable:"false"` WatchTimeoutInterval ParamItem `refreshable:"false"`
EnableBalanceChannelWithRPC ParamItem `refreshable:"false"`
LegacyVersionWithoutRPCWatch ParamItem `refreshable:"false"` LegacyVersionWithoutRPCWatch ParamItem `refreshable:"false"`
ChannelBalanceSilentDuration ParamItem `refreshable:"true"` ChannelBalanceSilentDuration ParamItem `refreshable:"true"`
ChannelBalanceInterval ParamItem `refreshable:"true"` ChannelBalanceInterval ParamItem `refreshable:"true"`
@ -2948,15 +2947,6 @@ func (p *dataCoordConfig) init(base *BaseTable) {
} }
p.WatchTimeoutInterval.Init(base.mgr) 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{ p.LegacyVersionWithoutRPCWatch = ParamItem{
Key: "dataCoord.channel.legacyVersionWithoutRPCWatch", Key: "dataCoord.channel.legacyVersionWithoutRPCWatch",
Version: "2.4.1", Version: "2.4.1",

View File

@ -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))
}