From e0fa6564e0daa63fcc12d51ea196aa8a106bee17 Mon Sep 17 00:00:00 2001 From: jaime Date: Wed, 19 Jun 2024 17:48:28 +0800 Subject: [PATCH] enhance: make subfunctions of datanode component modular Signed-off-by: jaime --- Makefile | 4 +- configs/milvus.yaml | 1 - go.mod | 4 +- internal/datacoord/channel.go | 13 +- internal/datacoord/channel_manager.go | 903 ------------ internal/datacoord/channel_manager_test.go | 1306 ----------------- internal/datacoord/channel_store.go | 5 +- internal/datacoord/channel_store_test.go | 53 - internal/datacoord/server.go | 13 +- .../datanode/{ => channel}/channel_manager.go | 126 +- .../{ => channel}/channel_manager_test.go | 111 +- .../{ => channel}/mock_channelmanager.go | 2 +- internal/datanode/compaction/l0_compactor.go | 3 +- .../{util => compaction}/load_stats.go | 2 +- internal/datanode/data_node.go | 149 +- internal/datanode/data_node_test.go | 100 +- internal/datanode/event_manager.go | 447 ------ internal/datanode/event_manager_test.go | 626 -------- internal/datanode/flow_graph_manager_test.go | 146 -- internal/datanode/flush_task_counter.go | 79 - internal/datanode/flush_task_counter_test.go | 44 - internal/datanode/metrics_info.go | 5 +- .../{ => pipeline}/data_sync_service.go | 174 +-- .../{ => pipeline}/data_sync_service_test.go | 290 ++-- .../{ => pipeline}/flow_graph_dd_node.go | 63 +- .../{ => pipeline}/flow_graph_dd_node_test.go | 111 +- .../flow_graph_dmstream_input_node.go | 6 +- .../flow_graph_dmstream_input_node_test.go | 2 +- .../{ => pipeline}/flow_graph_manager.go | 52 +- .../pipeline/flow_graph_manager_test.go | 130 ++ .../{ => pipeline}/flow_graph_message.go | 41 +- .../{ => pipeline}/flow_graph_message_test.go | 8 +- .../{ => pipeline}/flow_graph_node.go | 2 +- .../flow_graph_time_tick_node.go | 19 +- .../{ => pipeline}/flow_graph_time_ticker.go | 25 +- .../{ => pipeline}/flow_graph_write_node.go | 45 +- .../datanode/{ => pipeline}/mock_fgmanager.go | 78 +- .../datanode/{ => pipeline}/stats_updater.go | 9 +- .../{ => pipeline}/stats_updater_test.go | 2 +- internal/datanode/services.go | 7 +- internal/datanode/services_test.go | 34 +- internal/datanode/{ => util}/cache.go | 10 +- internal/datanode/{ => util}/cache_test.go | 4 +- .../checkpoint_updater.go} | 24 +- .../checkpoint_updater_test.go} | 10 +- internal/datanode/{ => util}/meta_util.go | 6 +- .../datanode/{ => util}/rate_collector.go | 43 +- .../{ => util}/rate_collector_test.go | 14 +- .../{mock_test.go => util/testutils.go} | 295 +--- internal/datanode/util/tickler.go | 51 + .../datanode/{ => util}/timetick_sender.go | 37 +- .../{ => util}/timetick_sender_test.go | 10 +- internal/datanode/{ => util}/util.go | 32 +- internal/mocks/mock_datanode.go | 90 +- internal/mocks/mock_datanode_client.go | 120 +- pkg/util/paramtable/component_param.go | 10 - .../watchcompatibility/watch_test.go | 365 ----- 57 files changed, 1024 insertions(+), 5337 deletions(-) delete mode 100644 internal/datacoord/channel_manager.go delete mode 100644 internal/datacoord/channel_manager_test.go rename internal/datanode/{ => channel}/channel_manager.go (81%) rename internal/datanode/{ => channel}/channel_manager_test.go (67%) rename internal/datanode/{ => channel}/mock_channelmanager.go (99%) rename internal/datanode/{util => compaction}/load_stats.go (99%) delete mode 100644 internal/datanode/event_manager.go delete mode 100644 internal/datanode/event_manager_test.go delete mode 100644 internal/datanode/flow_graph_manager_test.go delete mode 100644 internal/datanode/flush_task_counter.go delete mode 100644 internal/datanode/flush_task_counter_test.go rename internal/datanode/{ => pipeline}/data_sync_service.go (54%) rename internal/datanode/{ => pipeline}/data_sync_service_test.go (67%) rename internal/datanode/{ => pipeline}/flow_graph_dd_node.go (82%) rename internal/datanode/{ => pipeline}/flow_graph_dd_node_test.go (85%) rename internal/datanode/{ => pipeline}/flow_graph_dmstream_input_node.go (95%) rename internal/datanode/{ => pipeline}/flow_graph_dmstream_input_node_test.go (99%) rename internal/datanode/{ => pipeline}/flow_graph_manager.go (63%) create mode 100644 internal/datanode/pipeline/flow_graph_manager_test.go rename internal/datanode/{ => pipeline}/flow_graph_message.go (67%) rename internal/datanode/{ => pipeline}/flow_graph_message_test.go (85%) rename internal/datanode/{ => pipeline}/flow_graph_node.go (98%) rename internal/datanode/{ => pipeline}/flow_graph_time_tick_node.go (89%) rename internal/datanode/{ => pipeline}/flow_graph_time_ticker.go (81%) rename internal/datanode/{ => pipeline}/flow_graph_write_node.go (69%) rename internal/datanode/{ => pipeline}/mock_fgmanager.go (79%) rename internal/datanode/{ => pipeline}/stats_updater.go (88%) rename internal/datanode/{ => pipeline}/stats_updater_test.go (98%) rename internal/datanode/{ => util}/cache.go (93%) rename internal/datanode/{ => util}/cache_test.go (96%) rename internal/datanode/{channel_checkpoint_updater.go => util/checkpoint_updater.go} (90%) rename internal/datanode/{channel_checkpoint_updater_test.go => util/checkpoint_updater_test.go} (94%) rename internal/datanode/{ => util}/meta_util.go (95%) rename internal/datanode/{ => util}/rate_collector.go (62%) rename internal/datanode/{ => util}/rate_collector_test.go (80%) rename internal/datanode/{mock_test.go => util/testutils.go} (79%) create mode 100644 internal/datanode/util/tickler.go rename internal/datanode/{ => util}/timetick_sender.go (80%) rename internal/datanode/{ => util}/timetick_sender_test.go (96%) rename internal/datanode/{ => util}/util.go (57%) delete mode 100644 tests/integration/watchcompatibility/watch_test.go diff --git a/Makefile b/Makefile index 2a3372bad4953..aa0096e9afee4 100644 --- a/Makefile +++ b/Makefile @@ -478,8 +478,8 @@ generate-mockery-datanode: getdeps $(INSTALL_PATH)/mockery --name=WriteBuffer --dir=$(PWD)/internal/datanode/writebuffer --output=$(PWD)/internal/datanode/writebuffer --filename=mock_write_buffer.go --with-expecter --structname=MockWriteBuffer --outpkg=writebuffer --inpackage $(INSTALL_PATH)/mockery --name=BufferManager --dir=$(PWD)/internal/datanode/writebuffer --output=$(PWD)/internal/datanode/writebuffer --filename=mock_mananger.go --with-expecter --structname=MockBufferManager --outpkg=writebuffer --inpackage $(INSTALL_PATH)/mockery --name=BinlogIO --dir=$(PWD)/internal/datanode/io --output=$(PWD)/internal/datanode/io --filename=mock_binlogio.go --with-expecter --structname=MockBinlogIO --outpkg=io --inpackage - $(INSTALL_PATH)/mockery --name=FlowgraphManager --dir=$(PWD)/internal/datanode --output=$(PWD)/internal/datanode --filename=mock_fgmanager.go --with-expecter --structname=MockFlowgraphManager --outpkg=datanode --inpackage - $(INSTALL_PATH)/mockery --name=ChannelManager --dir=$(PWD)/internal/datanode --output=$(PWD)/internal/datanode --filename=mock_channelmanager.go --with-expecter --structname=MockChannelManager --outpkg=datanode --inpackage + $(INSTALL_PATH)/mockery --name=FlowgraphManager --dir=$(PWD)/internal/datanode/pipeline --output=$(PWD)/internal/datanode/pipeline --filename=mock_fgmanager.go --with-expecter --structname=MockFlowgraphManager --outpkg=pipeline --inpackage + $(INSTALL_PATH)/mockery --name=ChannelManager --dir=$(PWD)/internal/datanode/channel --output=$(PWD)/internal/datanode/channel --filename=mock_channelmanager.go --with-expecter --structname=MockChannelManager --outpkg=channel --inpackage $(INSTALL_PATH)/mockery --name=Compactor --dir=$(PWD)/internal/datanode/compaction --output=$(PWD)/internal/datanode/compaction --filename=mock_compactor.go --with-expecter --structname=MockCompactor --outpkg=compaction --inpackage generate-mockery-metastore: getdeps diff --git a/configs/milvus.yaml b/configs/milvus.yaml index 190b4e4d96a80..cd46bb8895250 100644 --- a/configs/milvus.yaml +++ b/configs/milvus.yaml @@ -419,7 +419,6 @@ indexNode: dataCoord: channel: watchTimeoutInterval: 300 # Timeout on watching channels (in seconds). Datanode tickler update watch progress will reset timeout timer. - balanceWithRpc: true # Whether to enable balance with RPC, default to use etcd watch legacyVersionWithoutRPCWatch: 2.4.0 # Datanodes <= this version are considered as legacy nodes, which doesn't have rpc based watch(). This is only used during rolling upgrade where legacy nodes won't get new channels balanceSilentDuration: 300 # The duration after which the channel manager start background channel balancing balanceInterval: 360 # The interval with which the channel manager check dml channel balance status diff --git a/go.mod b/go.mod index 653c0798af391..497dec8d09f2c 100644 --- a/go.mod +++ b/go.mod @@ -69,7 +69,9 @@ require ( github.com/jolestar/go-commons-pool/v2 v2.1.2 github.com/milvus-io/milvus/pkg v0.0.0-00010101000000-000000000000 github.com/pkg/errors v0.9.1 + github.com/valyala/fastjson v1.6.4 github.com/zeebo/xxh3 v1.0.2 + google.golang.org/protobuf v1.33.0 gopkg.in/yaml.v3 v3.0.1 ) @@ -208,7 +210,6 @@ require ( github.com/twmb/murmur3 v1.1.3 // indirect github.com/uber/jaeger-client-go v2.30.0+incompatible // indirect github.com/ugorji/go/codec v1.2.11 // indirect - github.com/valyala/fastjson v1.6.4 // indirect github.com/x448/float16 v0.8.4 // indirect github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2 // indirect github.com/yusufpapurcu/wmi v1.2.2 // indirect @@ -238,7 +239,6 @@ require ( google.golang.org/genproto v0.0.0-20230706204954-ccb25ca9f130 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20230629202037-9506855d4529 // indirect google.golang.org/genproto/googleapis/rpc v0.0.0-20240401170217-c3f982113cda // indirect - google.golang.org/protobuf v1.33.0 // indirect gopkg.in/inf.v0 v0.9.1 // indirect gopkg.in/ini.v1 v1.67.0 // indirect gopkg.in/natefinch/lumberjack.v2 v2.0.0 // indirect diff --git a/internal/datacoord/channel.go b/internal/datacoord/channel.go index f2a8f44bf8ba1..024c06d947669 100644 --- a/internal/datacoord/channel.go +++ b/internal/datacoord/channel.go @@ -26,7 +26,6 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/util/paramtable" ) type ROChannel interface { @@ -50,17 +49,7 @@ func NewRWChannel(name string, schema *schemapb.CollectionSchema, createTs uint64, ) RWChannel { - if paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.GetAsBool() { - return &StateChannel{ - Name: name, - CollectionID: collectionID, - StartPositions: startPos, - Schema: schema, - CreateTimestamp: createTs, - } - } - - return &channelMeta{ + return &StateChannel{ Name: name, CollectionID: collectionID, StartPositions: startPos, diff --git a/internal/datacoord/channel_manager.go b/internal/datacoord/channel_manager.go deleted file mode 100644 index 4fa1927660195..0000000000000 --- a/internal/datacoord/channel_manager.go +++ /dev/null @@ -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/kv" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/pkg/common" - "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) -} diff --git a/internal/datacoord/channel_manager_test.go b/internal/datacoord/channel_manager_test.go deleted file mode 100644 index 5866d638bc83a..0000000000000 --- a/internal/datacoord/channel_manager_test.go +++ /dev/null @@ -1,1306 +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" - "path" - "strconv" - "sync" - "testing" - "time" - - "github.com/golang/protobuf/proto" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - "go.uber.org/atomic" - - "github.com/milvus-io/milvus/internal/kv" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/util/dependency" - "github.com/milvus-io/milvus/pkg/common" - "github.com/milvus-io/milvus/pkg/util/paramtable" -) - -// waitAndStore simulates DataNode's action -func waitAndStore(t *testing.T, watchkv kv.MetaKv, key string, waitState, storeState datapb.ChannelWatchState) { - for { - v, err := watchkv.Load(key) - if err == nil && len(v) > 0 { - watchInfo, err := parseWatchInfo(key, []byte(v)) - require.NoError(t, err) - require.Equal(t, waitState, watchInfo.GetState()) - - watchInfo.State = storeState - data, err := proto.Marshal(watchInfo) - require.NoError(t, err) - - watchkv.Save(key, string(data)) - break - } - time.Sleep(100 * time.Millisecond) - } -} - -func waitPrefixAndStore(t *testing.T, watchkv kv.MetaKv, prefix string, waitState, storeState datapb.ChannelWatchState) string { - channelName := "" - for { - keys, values, err := watchkv.LoadWithPrefix(prefix) - if err == nil && len(values) > 0 { - for idx, value := range values { - watchInfo, err := parseWatchInfo(keys[idx], []byte(value)) - require.NoError(t, err) - require.Equal(t, waitState, watchInfo.GetState()) - - channelName = watchInfo.GetVchan().GetChannelName() - - watchInfo.State = storeState - data, err := proto.Marshal(watchInfo) - require.NoError(t, err) - - watchkv.Save(path.Join(prefix, watchInfo.GetVchan().GetChannelName()), string(data)) - } - break - } - time.Sleep(100 * time.Millisecond) - } - return channelName -} - -// waitAndCheckState checks if the DataCoord writes expected state into Etcd -func waitAndCheckState(t *testing.T, kv kv.MetaKv, expectedState datapb.ChannelWatchState, nodeID UniqueID, channelName string, collectionID UniqueID) { - for { - prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue() - v, err := kv.Load(path.Join(prefix, strconv.FormatInt(nodeID, 10), channelName)) - if err == nil && len(v) > 0 { - watchInfo, err := parseWatchInfo("fake", []byte(v)) - require.NoError(t, err) - - if watchInfo.GetState() == expectedState { - assert.Equal(t, watchInfo.Vchan.GetChannelName(), channelName) - assert.Equal(t, watchInfo.Vchan.GetCollectionID(), collectionID) - break - } - } - time.Sleep(100 * time.Millisecond) - } -} - -func getTestOps(nodeID UniqueID, ch RWChannel) *ChannelOpSet { - return NewChannelOpSet(NewAddOp(nodeID, ch)) -} - -func TestChannelManager_StateTransfer(t *testing.T) { - watchkv := getWatchKV(t) - defer func() { - watchkv.RemoveWithPrefix("") - watchkv.Close() - }() - - p := "/tmp/milvus_ut/rdb_data" - t.Setenv("ROCKSMQ_PATH", p) - - prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue() - - var ( - collectionID = UniqueID(9) - nodeID = UniqueID(119) - channelNamePrefix = t.Name() - - waitFor = time.Second * 10 - tick = time.Millisecond * 10 - ) - - t.Run("ToWatch-WatchSuccess", func(t *testing.T) { - watchkv.RemoveWithPrefix("") - cName := channelNamePrefix + "ToWatch-WatchSuccess" - - ctx, cancel := context.WithCancel(context.TODO()) - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - - wg := sync.WaitGroup{} - wg.Add(1) - go func() { - chManager.watchChannelStatesLoop(ctx, common.LatestRevision) - wg.Done() - }() - - chManager.AddNode(nodeID) - chManager.Watch(ctx, &channelMeta{Name: cName, CollectionID: collectionID}) - - key := buildNodeChannelKey(nodeID, cName) - waitAndStore(t, watchkv, key, datapb.ChannelWatchState_ToWatch, datapb.ChannelWatchState_WatchSuccess) - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_WatchSuccess, nodeID, cName, collectionID) - - assert.Eventually(t, func() bool { - loaded := chManager.stateTimer.runningTimerStops.Contain(cName) - return !loaded - }, waitFor, tick) - - cancel() - wg.Wait() - }) - - t.Run("ToWatch-WatchFail-ToRelease", func(t *testing.T) { - watchkv.RemoveWithPrefix("") - cName := channelNamePrefix + "ToWatch-WatchFail-ToRelase" - ctx, cancel := context.WithCancel(context.TODO()) - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - - wg := sync.WaitGroup{} - wg.Add(1) - go func() { - chManager.watchChannelStatesLoop(ctx, common.LatestRevision) - wg.Done() - }() - - chManager.AddNode(nodeID) - chManager.Watch(ctx, &channelMeta{Name: cName, CollectionID: collectionID}) - - key := path.Join(prefix, strconv.FormatInt(nodeID, 10), cName) - waitAndStore(t, watchkv, key, datapb.ChannelWatchState_ToWatch, datapb.ChannelWatchState_WatchFailure) - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToRelease, nodeID, cName, collectionID) - - assert.Eventually(t, func() bool { - loaded := chManager.stateTimer.runningTimerStops.Contain(cName) - return loaded - }, waitFor, tick) - - cancel() - wg.Wait() - chManager.stateTimer.removeTimers([]string{cName}) - }) - - t.Run("ToWatch-Timeout", func(t *testing.T) { - watchkv.RemoveWithPrefix("") - cName := channelNamePrefix + "ToWatch-Timeout" - ctx, cancel := context.WithCancel(context.TODO()) - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - - wg := sync.WaitGroup{} - wg.Add(1) - go func() { - chManager.watchChannelStatesLoop(ctx, common.LatestRevision) - wg.Done() - }() - - chManager.AddNode(nodeID) - chManager.Watch(ctx, &channelMeta{Name: cName, CollectionID: collectionID}) - - // simulating timeout behavior of startOne, cuz 20s is a long wait - e := &ackEvent{ - ackType: watchTimeoutAck, - channelName: cName, - nodeID: nodeID, - } - chManager.stateTimer.notifyTimeoutWatcher(e) - - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToRelease, nodeID, cName, collectionID) - assert.Eventually(t, func() bool { - loaded := chManager.stateTimer.runningTimerStops.Contain(cName) - return loaded - }, waitFor, tick) - - cancel() - wg.Wait() - chManager.stateTimer.removeTimers([]string{cName}) - }) - - t.Run("ToRelease-ReleaseSuccess-Reassign-ToWatch-2-DN", func(t *testing.T) { - oldNode := UniqueID(120) - cName := channelNamePrefix + "ToRelease-ReleaseSuccess-Reassign-ToWatch-2-DN" - - watchkv.RemoveWithPrefix("") - ctx, cancel := context.WithCancel(context.TODO()) - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - - wg := sync.WaitGroup{} - wg.Add(1) - go func() { - chManager.watchChannelStatesLoop(ctx, common.LatestRevision) - wg.Done() - }() - - chManager.store = &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - nodeID: NewNodeChannelInfo(nodeID, &channelMeta{Name: cName, CollectionID: collectionID}), - oldNode: NewNodeChannelInfo(oldNode), - }, - } - - err = chManager.Release(nodeID, cName) - assert.NoError(t, err) - - key := path.Join(prefix, strconv.FormatInt(nodeID, 10), cName) - waitAndStore(t, watchkv, key, datapb.ChannelWatchState_ToRelease, datapb.ChannelWatchState_ReleaseSuccess) - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, oldNode, cName, collectionID) - - cancel() - wg.Wait() - - w, err := watchkv.Load(path.Join(prefix, strconv.FormatInt(nodeID, 10))) - assert.Error(t, err) - assert.Empty(t, w) - - loaded := chManager.stateTimer.runningTimerStops.Contain(cName) - assert.True(t, loaded) - chManager.stateTimer.removeTimers([]string{cName}) - }) - - t.Run("ToRelease-ReleaseSuccess-Reassign-ToWatch-1-DN", func(t *testing.T) { - watchkv.RemoveWithPrefix("") - ctx, cancel := context.WithCancel(context.TODO()) - cName := channelNamePrefix + "ToRelease-ReleaseSuccess-Reassign-ToWatch-1-DN" - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - - wg := sync.WaitGroup{} - wg.Add(1) - go func() { - chManager.watchChannelStatesLoop(ctx, common.LatestRevision) - wg.Done() - }() - - chManager.store = &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - nodeID: NewNodeChannelInfo(nodeID, &channelMeta{Name: cName, CollectionID: collectionID}), - }, - } - - err = chManager.Release(nodeID, cName) - assert.NoError(t, err) - - key := path.Join(prefix, strconv.FormatInt(nodeID, 10), cName) - waitAndStore(t, watchkv, key, datapb.ChannelWatchState_ToRelease, datapb.ChannelWatchState_ReleaseSuccess) - - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, nodeID, cName, collectionID) - - assert.Eventually(t, func() bool { - loaded := chManager.stateTimer.runningTimerStops.Contain(cName) - return loaded - }, waitFor, tick) - cancel() - wg.Wait() - - chManager.stateTimer.removeTimers([]string{cName}) - }) - - t.Run("ToRelease-ReleaseFail-CleanUpAndDelete-Reassign-ToWatch-2-DN", func(t *testing.T) { - oldNode := UniqueID(121) - - cName := channelNamePrefix + "ToRelease-ReleaseFail-CleanUpAndDelete-Reassign-ToWatch-2-DN" - watchkv.RemoveWithPrefix("") - ctx, cancel := context.WithCancel(context.TODO()) - factory := dependency.NewDefaultFactory(true) - _, err := factory.NewMsgStream(context.TODO()) - require.NoError(t, err) - chManager, err := NewChannelManager(watchkv, newMockHandler(), withMsgstreamFactory(factory)) - require.NoError(t, err) - - wg := sync.WaitGroup{} - wg.Add(1) - go func() { - chManager.watchChannelStatesLoop(ctx, common.LatestRevision) - wg.Done() - }() - - chManager.store = &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - nodeID: NewNodeChannelInfo(nodeID, &channelMeta{Name: cName, CollectionID: collectionID}), - oldNode: NewNodeChannelInfo(oldNode), - }, - } - - err = chManager.Release(nodeID, cName) - assert.NoError(t, err) - - key := path.Join(prefix, strconv.FormatInt(nodeID, 10), cName) - waitAndStore(t, watchkv, key, datapb.ChannelWatchState_ToRelease, datapb.ChannelWatchState_ReleaseFailure) - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, oldNode, cName, collectionID) - - cancel() - wg.Wait() - - w, err := watchkv.Load(path.Join(prefix, strconv.FormatInt(nodeID, 10))) - assert.Error(t, err) - assert.Empty(t, w) - - loaded := chManager.stateTimer.runningTimerStops.Contain(cName) - assert.True(t, loaded) - chManager.stateTimer.removeTimers([]string{cName}) - }) - - t.Run("ToRelease-ReleaseFail-CleanUpAndDelete-Reassign-ToWatch-1-DN", func(t *testing.T) { - watchkv.RemoveWithPrefix("") - cName := channelNamePrefix + "ToRelease-ReleaseFail-CleanUpAndDelete-Reassign-ToWatch-1-DN" - ctx, cancel := context.WithCancel(context.TODO()) - factory := dependency.NewDefaultFactory(true) - _, err := factory.NewMsgStream(context.TODO()) - require.NoError(t, err) - chManager, err := NewChannelManager(watchkv, newMockHandler(), withMsgstreamFactory(factory)) - require.NoError(t, err) - - wg := sync.WaitGroup{} - wg.Add(1) - go func() { - chManager.watchChannelStatesLoop(ctx, common.LatestRevision) - wg.Done() - }() - - chManager.store = &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - nodeID: NewNodeChannelInfo(nodeID, &channelMeta{Name: cName, CollectionID: collectionID}), - }, - } - - err = chManager.Release(nodeID, cName) - assert.NoError(t, err) - - key := path.Join(prefix, strconv.FormatInt(nodeID, 10), cName) - waitAndStore(t, watchkv, key, datapb.ChannelWatchState_ToRelease, datapb.ChannelWatchState_ReleaseFailure) - - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, nodeID, cName, collectionID) - assert.Eventually(t, func() bool { - loaded := chManager.stateTimer.runningTimerStops.Contain(cName) - return loaded - }, waitFor, tick) - - cancel() - wg.Wait() - chManager.stateTimer.removeTimers([]string{cName}) - }) -} - -func TestChannelManager(t *testing.T) { - watchkv := getWatchKV(t) - defer func() { - watchkv.RemoveWithPrefix("") - watchkv.Close() - }() - - Params.Save(Params.DataCoordCfg.AutoBalance.Key, "true") - prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue() - - enableRPCK := paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.Key - paramtable.Get().Save(enableRPCK, "false") - defer paramtable.Get().Reset(enableRPCK) - t.Run("test AddNode with avalible node", func(t *testing.T) { - // Note: this test is based on the default registerPolicy - defer watchkv.RemoveWithPrefix("") - var ( - collectionID = UniqueID(8) - nodeID, nodeToAdd = UniqueID(118), UniqueID(811) - channel1, channel2 = "channel1", "channel2" - ) - - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - chManager.store = &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - nodeID: NewNodeChannelInfo(nodeID, &channelMeta{Name: channel1, CollectionID: collectionID}, &channelMeta{Name: channel2, CollectionID: collectionID}), - }, - } - - err = chManager.AddNode(nodeToAdd) - assert.NoError(t, err) - - assert.True(t, chManager.Match(nodeID, channel1)) - assert.True(t, chManager.Match(nodeID, channel2)) - assert.False(t, chManager.Match(nodeToAdd, channel1)) - assert.False(t, chManager.Match(nodeToAdd, channel2)) - - err = chManager.Watch(context.TODO(), &channelMeta{Name: "channel-3", CollectionID: collectionID}) - assert.NoError(t, err) - - assert.True(t, chManager.Match(nodeToAdd, "channel-3")) - - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, nodeToAdd, "channel-3", collectionID) - chManager.stateTimer.removeTimers([]string{"channel-3"}) - }) - - t.Run("test AddNode with no available node", func(t *testing.T) { - // Note: this test is based on the default registerPolicy - defer watchkv.RemoveWithPrefix("") - var ( - collectionID = UniqueID(8) - nodeID = UniqueID(119) - channel1, channel2 = "channel1", "channel2" - ) - - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - chManager.store = &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - bufferID: NewNodeChannelInfo(bufferID, &channelMeta{Name: channel1, CollectionID: collectionID}, &channelMeta{Name: channel2, CollectionID: collectionID}), - }, - } - - err = chManager.AddNode(nodeID) - assert.NoError(t, err) - - key := path.Join(prefix, strconv.FormatInt(nodeID, 10), channel1) - waitAndStore(t, watchkv, key, datapb.ChannelWatchState_ToWatch, datapb.ChannelWatchState_WatchSuccess) - - key = path.Join(prefix, strconv.FormatInt(nodeID, 10), channel2) - waitAndStore(t, watchkv, key, datapb.ChannelWatchState_ToWatch, datapb.ChannelWatchState_WatchSuccess) - - assert.True(t, chManager.Match(nodeID, channel1)) - assert.True(t, chManager.Match(nodeID, channel2)) - - err = chManager.Watch(context.TODO(), &channelMeta{Name: "channel-3", CollectionID: collectionID}) - assert.NoError(t, err) - - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, nodeID, "channel-3", collectionID) - chManager.stateTimer.removeTimers([]string{"channel-3"}) - }) - - t.Run("test Watch", func(t *testing.T) { - defer watchkv.RemoveWithPrefix("") - var ( - collectionID = UniqueID(7) - nodeID = UniqueID(117) - bufferCh = "bufferID" - chanToAdd = "new-channel-watch" - ) - - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - - err = chManager.Watch(context.TODO(), &channelMeta{Name: bufferCh, CollectionID: collectionID}) - assert.NoError(t, err) - - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, bufferID, bufferCh, collectionID) - - chManager.store.AddNode(nodeID) - err = chManager.Watch(context.TODO(), &channelMeta{Name: chanToAdd, CollectionID: collectionID}) - assert.NoError(t, err) - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, nodeID, chanToAdd, collectionID) - - chManager.stateTimer.removeTimers([]string{chanToAdd}) - }) - - t.Run("test Release", func(t *testing.T) { - defer watchkv.RemoveWithPrefix("") - var ( - collectionID = UniqueID(4) - nodeID, invalidNodeID = UniqueID(114), UniqueID(999) - channelName, invalidChName = "to-release", "invalid-to-release" - ) - - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - chManager.store = &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - nodeID: NewNodeChannelInfo(nodeID, &channelMeta{Name: channelName, CollectionID: collectionID}), - }, - } - - err = chManager.Release(invalidNodeID, invalidChName) - assert.Error(t, err) - - err = chManager.Release(nodeID, channelName) - assert.NoError(t, err) - chManager.stateTimer.removeTimers([]string{channelName}) - - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToRelease, nodeID, channelName, collectionID) - }) - - t.Run("test Reassign", func(t *testing.T) { - defer watchkv.RemoveWithPrefix("") - collectionID := UniqueID(5) - - tests := []struct { - nodeID UniqueID - chName string - }{ - {UniqueID(125), "normal-chan"}, - {UniqueID(115), "to-delete-chan"}, - } - - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - - // prepare tests - for _, test := range tests { - chManager.store.AddNode(test.nodeID) - ops := getTestOps(test.nodeID, &channelMeta{Name: test.chName, CollectionID: collectionID, WatchInfo: &datapb.ChannelWatchInfo{}}) - err = chManager.store.Update(ops) - require.NoError(t, err) - - info, err := watchkv.Load(path.Join(prefix, strconv.FormatInt(test.nodeID, 10), test.chName)) - require.NoError(t, err) - require.NotNil(t, info) - } - - remainTest, reassignTest := tests[0], tests[1] - err = chManager.Reassign(reassignTest.nodeID, reassignTest.chName) - assert.NoError(t, err) - chManager.stateTimer.stopIfExist(&ackEvent{watchSuccessAck, reassignTest.chName, reassignTest.nodeID}) - - // test nodes of reassignTest contains no channel - // test all channels are assgined to node of remainTest - assert.False(t, chManager.Match(reassignTest.nodeID, reassignTest.chName)) - assert.True(t, chManager.Match(remainTest.nodeID, reassignTest.chName)) - assert.True(t, chManager.Match(remainTest.nodeID, remainTest.chName)) - - // Delete node of reassginTest and try to Reassign node in remainTest - err = chManager.DeleteNode(reassignTest.nodeID) - require.NoError(t, err) - - err = chManager.Reassign(remainTest.nodeID, remainTest.chName) - assert.NoError(t, err) - chManager.stateTimer.stopIfExist(&ackEvent{releaseSuccessAck, reassignTest.chName, reassignTest.nodeID}) - - // channel is added to remainTest because there's only one node left - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, remainTest.nodeID, remainTest.chName, collectionID) - }) - - t.Run("test Reassign with get channel fail", func(t *testing.T) { - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - - err = chManager.Reassign(1, "not-exists-channelName") - assert.Error(t, err) - }) - - t.Run("test Reassign with dropped channel", func(t *testing.T) { - collectionID := UniqueID(5) - watchkv.RemoveWithPrefix("") - handler := NewNMockHandler(t) - handler.EXPECT(). - CheckShouldDropChannel(mock.Anything). - Return(true) - handler.EXPECT().FinishDropChannel(mock.Anything, mock.Anything).Return(nil) - chManager, err := NewChannelManager(watchkv, handler) - require.NoError(t, err) - - chManager.store.AddNode(1) - ops := getTestOps(1, &channelMeta{Name: "chan", CollectionID: collectionID, WatchInfo: &datapb.ChannelWatchInfo{}}) - err = chManager.store.Update(ops) - require.NoError(t, err) - - assert.Equal(t, 1, chManager.store.GetNodeChannelCount(1)) - err = chManager.Reassign(1, "chan") - assert.NoError(t, err) - assert.Equal(t, 0, chManager.store.GetNodeChannelCount(1)) - }) - - t.Run("test Reassign-channel not found", func(t *testing.T) { - var chManager *ChannelManagerImpl - var err error - handler := NewNMockHandler(t) - chManager, err = NewChannelManager(watchkv, handler) - require.NoError(t, err) - - chManager.store.AddNode(1) - ops := getTestOps(1, &channelMeta{Name: "chan", CollectionID: 1, WatchInfo: &datapb.ChannelWatchInfo{}}) - err = chManager.store.Update(ops) - require.NoError(t, err) - - assert.Equal(t, 1, chManager.store.GetNodeChannelCount(1)) - err = chManager.Reassign(2, "chan") - assert.Error(t, err) - }) - - t.Run("test CleanupAndReassign-channel not found", func(t *testing.T) { - var chManager *ChannelManagerImpl - var err error - handler := NewNMockHandler(t) - - watchkv.RemoveWithPrefix("") - chManager, err = NewChannelManager(watchkv, handler) - require.NoError(t, err) - - chManager.store.AddNode(1) - ops := getTestOps(1, &channelMeta{Name: "chan", CollectionID: 1, WatchInfo: &datapb.ChannelWatchInfo{}}) - err = chManager.store.Update(ops) - require.NoError(t, err) - - assert.Equal(t, 1, chManager.store.GetNodeChannelCount(1)) - err = chManager.CleanupAndReassign(2, "chan") - assert.Error(t, err) - }) - - t.Run("test CleanupAndReassign with get channel fail", func(t *testing.T) { - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - - err = chManager.CleanupAndReassign(1, "not-exists-channelName") - assert.Error(t, err) - }) - - t.Run("test CleanupAndReassign with dropped channel", func(t *testing.T) { - handler := NewNMockHandler(t) - handler.EXPECT(). - CheckShouldDropChannel(mock.Anything). - Return(true) - handler.EXPECT().FinishDropChannel(mock.Anything, mock.Anything).Return(nil) - watchkv.RemoveWithPrefix("") - chManager, err := NewChannelManager(watchkv, handler) - require.NoError(t, err) - - chManager.store.AddNode(1) - ops := getTestOps(1, &channelMeta{Name: "chan", CollectionID: 1, WatchInfo: &datapb.ChannelWatchInfo{}}) - err = chManager.store.Update(ops) - require.NoError(t, err) - - assert.Equal(t, 1, chManager.store.GetNodeChannelCount(1)) - err = chManager.CleanupAndReassign(1, "chan") - assert.NoError(t, err) - assert.Equal(t, 0, chManager.store.GetNodeChannelCount(1)) - }) - - t.Run("test DeleteNode", func(t *testing.T) { - defer watchkv.RemoveWithPrefix("") - - collectionID := UniqueID(999) - chManager, err := NewChannelManager(watchkv, newMockHandler(), withStateChecker()) - require.NoError(t, err) - chManager.store = &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - 1: NewNodeChannelInfo(1, &channelMeta{Name: "channel-1", CollectionID: collectionID}, - &channelMeta{Name: "channel-2", CollectionID: collectionID}), - bufferID: NewNodeChannelInfo(bufferID), - }, - } - chManager.stateTimer.startOne(datapb.ChannelWatchState_ToRelease, "channel-1", 1, Params.DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second)) - - err = chManager.DeleteNode(1) - assert.NoError(t, err) - - chs := chManager.store.GetBufferChannelInfo() - assert.Equal(t, 2, len(chs.Channels)) - }) - - t.Run("test CleanupAndReassign", func(t *testing.T) { - defer watchkv.RemoveWithPrefix("") - collectionID := UniqueID(6) - - tests := []struct { - nodeID UniqueID - chName string - }{ - {UniqueID(126), "normal-chan"}, - {UniqueID(116), "to-delete-chan"}, - } - - factory := dependency.NewDefaultFactory(true) - _, err := factory.NewMsgStream(context.TODO()) - require.NoError(t, err) - chManager, err := NewChannelManager(watchkv, newMockHandler(), withMsgstreamFactory(factory)) - - require.NoError(t, err) - - // prepare tests - for _, test := range tests { - chManager.store.AddNode(test.nodeID) - ops := getTestOps(test.nodeID, &channelMeta{Name: test.chName, CollectionID: collectionID, WatchInfo: &datapb.ChannelWatchInfo{}}) - err = chManager.store.Update(ops) - require.NoError(t, err) - - info, err := watchkv.Load(path.Join(prefix, strconv.FormatInt(test.nodeID, 10), test.chName)) - require.NoError(t, err) - require.NotNil(t, info) - } - - remainTest, reassignTest := tests[0], tests[1] - err = chManager.CleanupAndReassign(reassignTest.nodeID, reassignTest.chName) - assert.NoError(t, err) - chManager.stateTimer.stopIfExist(&ackEvent{releaseSuccessAck, reassignTest.chName, reassignTest.nodeID}) - - // test nodes of reassignTest contains no channel - assert.False(t, chManager.Match(reassignTest.nodeID, reassignTest.chName)) - - // test all channels are assgined to node of remainTest - assert.True(t, chManager.Match(remainTest.nodeID, reassignTest.chName)) - assert.True(t, chManager.Match(remainTest.nodeID, remainTest.chName)) - - // Delete node of reassginTest and try to CleanupAndReassign node in remainTest - err = chManager.DeleteNode(reassignTest.nodeID) - require.NoError(t, err) - - err = chManager.CleanupAndReassign(remainTest.nodeID, remainTest.chName) - assert.NoError(t, err) - chManager.stateTimer.stopIfExist(&ackEvent{releaseSuccessAck, reassignTest.chName, reassignTest.nodeID}) - - // channel is added to remainTest because there's only one node left - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, remainTest.nodeID, remainTest.chName, collectionID) - }) - - t.Run("test getChannelByNodeAndName", func(t *testing.T) { - defer watchkv.RemoveWithPrefix("") - var ( - nodeID = UniqueID(113) - collectionID = UniqueID(3) - channelName = "get-channel-by-node-and-name" - ) - - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - - ch := chManager.getChannelByNodeAndName(nodeID, channelName) - assert.Nil(t, ch) - - chManager.store.AddNode(nodeID) - ch = chManager.getChannelByNodeAndName(nodeID, channelName) - assert.Nil(t, ch) - - chManager.store = &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - nodeID: NewNodeChannelInfo(nodeID, &channelMeta{Name: channelName, CollectionID: collectionID}), - }, - } - ch = chManager.getChannelByNodeAndName(nodeID, channelName) - assert.NotNil(t, ch) - assert.Equal(t, collectionID, ch.GetCollectionID()) - assert.Equal(t, channelName, ch.GetName()) - }) - - t.Run("test fillChannelWatchInfoWithState", func(t *testing.T) { - defer watchkv.RemoveWithPrefix("") - var ( - nodeID = UniqueID(111) - collectionID = UniqueID(1) - channelName = "fill-channel-watchInfo-with-state" - ) - - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - - tests := []struct { - inState datapb.ChannelWatchState - - description string - }{ - {datapb.ChannelWatchState_ToWatch, "fill toWatch state"}, - {datapb.ChannelWatchState_ToRelease, "fill toRelase state"}, - } - - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - ops := NewChannelOpSet(NewAddOp(nodeID, &channelMeta{Name: channelName, CollectionID: collectionID})) - for _, op := range ops.Collect() { - chs := chManager.fillChannelWatchInfoWithState(op, test.inState) - assert.Equal(t, 1, len(chs)) - assert.Equal(t, channelName, chs[0]) - assert.NotNil(t, op.Channels[0].GetWatchInfo()) - assert.Equal(t, test.inState, op.Channels[0].GetWatchInfo().GetState()) - - chManager.stateTimer.removeTimers(chs) - } - }) - } - }) - - t.Run("test updateWithTimer", func(t *testing.T) { - var ( - nodeID = UniqueID(112) - collectionID = UniqueID(2) - channelName = "update-with-timer" - ) - - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - chManager.store.AddNode(nodeID) - - opSet := NewChannelOpSet(NewAddOp(nodeID, &channelMeta{Name: channelName, CollectionID: collectionID})) - - chManager.updateWithTimer(opSet, datapb.ChannelWatchState_ToWatch) - chManager.stateTimer.removeTimers([]string{channelName}) - - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, nodeID, channelName, collectionID) - }) - - t.Run("test background check silent", func(t *testing.T) { - watchkv.RemoveWithPrefix("") - defer watchkv.RemoveWithPrefix("") - prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue() - var ( - collectionID = UniqueID(9) - channelNamePrefix = t.Name() - nodeID = UniqueID(111) - ) - cName := channelNamePrefix + "TestBgChecker" - - // 1. set up channel_manager - ctx, cancel := context.WithCancel(context.TODO()) - defer cancel() - chManager, err := NewChannelManager(watchkv, newMockHandler(), withBgChecker()) - require.NoError(t, err) - assert.NotNil(t, chManager.bgChecker) - chManager.Startup(ctx, nil, []int64{nodeID}) - - // 2. test isSilent function running correctly - Params.Save(Params.DataCoordCfg.ChannelBalanceSilentDuration.Key, "3") - assert.False(t, chManager.isSilent()) - assert.False(t, chManager.stateTimer.hasRunningTimers()) - - // 3. watch one channel - chManager.Watch(ctx, &channelMeta{Name: cName, CollectionID: collectionID}) - assert.False(t, chManager.isSilent()) - assert.True(t, chManager.stateTimer.hasRunningTimers()) - key := path.Join(prefix, strconv.FormatInt(nodeID, 10), cName) - waitAndStore(t, watchkv, key, datapb.ChannelWatchState_ToWatch, datapb.ChannelWatchState_WatchSuccess) - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_WatchSuccess, nodeID, cName, collectionID) - - // 4. wait for duration and check silent again - time.Sleep(Params.DataCoordCfg.ChannelBalanceSilentDuration.GetAsDuration(time.Second)) - chManager.stateTimer.removeTimers([]string{cName}) - assert.True(t, chManager.isSilent()) - assert.False(t, chManager.stateTimer.hasRunningTimers()) - }) -} - -func TestChannelManager_Reload(t *testing.T) { - watchkv := getWatchKV(t) - defer func() { - watchkv.RemoveWithPrefix("") - watchkv.Close() - }() - - var ( - nodeID = UniqueID(200) - collectionID = UniqueID(2) - channelName = "channel-checkOldNodes" - ) - prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue() - - getWatchInfoWithState := func(state datapb.ChannelWatchState, collectionID UniqueID, channelName string) *datapb.ChannelWatchInfo { - return &datapb.ChannelWatchInfo{ - Vchan: &datapb.VchannelInfo{ - CollectionID: collectionID, - ChannelName: channelName, - }, - State: state, - } - } - - t.Run("test checkOldNodes", func(t *testing.T) { - watchkv.RemoveWithPrefix("") - - t.Run("ToWatch", func(t *testing.T) { - defer watchkv.RemoveWithPrefix("") - data, err := proto.Marshal(getWatchInfoWithState(datapb.ChannelWatchState_ToWatch, collectionID, channelName)) - require.NoError(t, err) - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - err = watchkv.Save(path.Join(prefix, strconv.FormatInt(nodeID, 10), channelName), string(data)) - require.NoError(t, err) - - chManager.checkOldNodes([]UniqueID{nodeID}) - ok := chManager.stateTimer.runningTimerStops.Contain(channelName) - assert.True(t, ok) - chManager.stateTimer.removeTimers([]string{channelName}) - }) - - t.Run("ToRelease", func(t *testing.T) { - defer watchkv.RemoveWithPrefix("") - data, err := proto.Marshal(getWatchInfoWithState(datapb.ChannelWatchState_ToRelease, collectionID, channelName)) - require.NoError(t, err) - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - err = watchkv.Save(path.Join(prefix, strconv.FormatInt(nodeID, 10), channelName), string(data)) - require.NoError(t, err) - err = chManager.checkOldNodes([]UniqueID{nodeID}) - assert.NoError(t, err) - - ok := chManager.stateTimer.runningTimerStops.Contain(channelName) - assert.True(t, ok) - chManager.stateTimer.removeTimers([]string{channelName}) - }) - - t.Run("WatchFail", func(t *testing.T) { - defer watchkv.RemoveWithPrefix("") - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - chManager.store = &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - nodeID: NewNodeChannelInfo(nodeID, &channelMeta{Name: channelName, CollectionID: collectionID}), - }, - } - - data, err := proto.Marshal(getWatchInfoWithState(datapb.ChannelWatchState_WatchFailure, collectionID, channelName)) - require.NoError(t, err) - err = watchkv.Save(path.Join(prefix, strconv.FormatInt(nodeID, 10), channelName), string(data)) - require.NoError(t, err) - err = chManager.checkOldNodes([]UniqueID{nodeID}) - assert.NoError(t, err) - - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToRelease, nodeID, channelName, collectionID) - chManager.stateTimer.removeTimers([]string{channelName}) - }) - - t.Run("ReleaseSuccess", func(t *testing.T) { - defer watchkv.RemoveWithPrefix("") - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - data, err := proto.Marshal(getWatchInfoWithState(datapb.ChannelWatchState_ReleaseSuccess, collectionID, channelName)) - chManager.store = &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - nodeID: NewNodeChannelInfo(nodeID, &channelMeta{Name: channelName, CollectionID: collectionID}), - }, - } - - require.NoError(t, err) - chManager.AddNode(UniqueID(111)) - err = watchkv.Save(path.Join(prefix, strconv.FormatInt(nodeID, 10), channelName), string(data)) - require.NoError(t, err) - err = chManager.checkOldNodes([]UniqueID{nodeID}) - assert.NoError(t, err) - - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, 111, channelName, collectionID) - chManager.stateTimer.removeTimers([]string{channelName}) - - v, err := watchkv.Load(path.Join(prefix, strconv.FormatInt(nodeID, 10))) - assert.Error(t, err) - assert.Empty(t, v) - }) - - t.Run("ReleaseFail", func(t *testing.T) { - defer watchkv.RemoveWithPrefix("") - chManager, err := NewChannelManager(watchkv, newMockHandler()) - require.NoError(t, err) - data, err := proto.Marshal(getWatchInfoWithState(datapb.ChannelWatchState_ReleaseFailure, collectionID, channelName)) - chManager.store = &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - nodeID: NewNodeChannelInfo(nodeID, &channelMeta{Name: channelName, CollectionID: collectionID}), - 999: NewNodeChannelInfo(999), - }, - } - require.NoError(t, err) - err = watchkv.Save(path.Join(prefix, strconv.FormatInt(nodeID, 10), channelName), string(data)) - require.NoError(t, err) - err = chManager.checkOldNodes([]UniqueID{nodeID}) - assert.NoError(t, err) - - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, 999, channelName, collectionID) - - v, err := watchkv.Load(path.Join(prefix, strconv.FormatInt(nodeID, 10), channelName)) - assert.Error(t, err) - assert.Empty(t, v) - }) - }) - - t.Run("test reload with data", func(t *testing.T) { - defer watchkv.RemoveWithPrefix("") - - ctx, cancel := context.WithCancel(context.TODO()) - defer cancel() - - cm, err := NewChannelManager(watchkv, newMockHandler()) - assert.NoError(t, err) - assert.Nil(t, cm.AddNode(1)) - assert.Nil(t, cm.AddNode(2)) - cm.store = &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - 1: NewNodeChannelInfo(1, &channelMeta{Name: "channel1", CollectionID: 1}), - 2: NewNodeChannelInfo(2, &channelMeta{Name: "channel2", CollectionID: 1}), - }, - } - - data, err := proto.Marshal(getWatchInfoWithState(datapb.ChannelWatchState_WatchSuccess, 1, "channel1")) - require.NoError(t, err) - err = watchkv.Save(path.Join(prefix, strconv.FormatInt(1, 10), "channel1"), string(data)) - require.NoError(t, err) - data, err = proto.Marshal(getWatchInfoWithState(datapb.ChannelWatchState_WatchSuccess, 1, "channel2")) - require.NoError(t, err) - err = watchkv.Save(path.Join(prefix, strconv.FormatInt(2, 10), "channel2"), string(data)) - require.NoError(t, err) - - cm2, err := NewChannelManager(watchkv, newMockHandler()) - assert.NoError(t, err) - assert.Nil(t, cm2.Startup(ctx, nil, []int64{3})) - - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, 3, "channel1", 1) - waitAndCheckState(t, watchkv, datapb.ChannelWatchState_ToWatch, 3, "channel2", 1) - assert.True(t, cm2.Match(3, "channel1")) - assert.True(t, cm2.Match(3, "channel2")) - - cm2.stateTimer.removeTimers([]string{"channel1", "channel2"}) - }) -} - -func TestChannelManager_BalanceBehaviour(t *testing.T) { - watchkv := getWatchKV(t) - defer func() { - watchkv.RemoveWithPrefix("") - watchkv.Close() - }() - - prefix := Params.CommonCfg.DataCoordWatchSubPath.GetValue() - - Params.Save(Params.DataCoordCfg.AutoBalance.Key, "true") - t.Run("one node with three channels add a new node", func(t *testing.T) { - defer watchkv.RemoveWithPrefix("") - - collectionID := UniqueID(999) - - chManager, err := NewChannelManager(watchkv, newMockHandler(), withStateChecker()) - require.NoError(t, err) - - ctx, cancel := context.WithCancel(context.TODO()) - chManager.stopChecker = cancel - defer cancel() - go chManager.stateChecker(ctx, common.LatestRevision) - - chManager.store = &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - 1: NewNodeChannelInfo(1, &channelMeta{Name: "channel-1", CollectionID: collectionID}, - &channelMeta{Name: "channel-2", CollectionID: collectionID}, - &channelMeta{Name: "channel-3", CollectionID: collectionID}), - }, - } - - var channelBalanced string - - chManager.AddNode(2) - - watchPrefix := path.Join(prefix, "1") - channelBalanced = waitPrefixAndStore(t, watchkv, watchPrefix, datapb.ChannelWatchState_ToRelease, datapb.ChannelWatchState_ReleaseSuccess) - - key := path.Join(prefix, "2", channelBalanced) - waitAndStore(t, watchkv, key, datapb.ChannelWatchState_ToWatch, datapb.ChannelWatchState_WatchSuccess) - - for _, channel := range []string{"channel-1", "channel-2", "channel-3"} { - if channel == channelBalanced { - assert.True(t, chManager.Match(2, channel)) - } else { - assert.True(t, chManager.Match(1, channel)) - } - } - - chManager.AddNode(3) - chManager.Watch(ctx, &channelMeta{Name: "channel-4", CollectionID: collectionID}) - // key = path.Join(prefix, "3", "channel-4") - watchPrefix = path.Join(prefix, "3") - channelBalanced2 := waitPrefixAndStore(t, watchkv, watchPrefix, datapb.ChannelWatchState_ToWatch, datapb.ChannelWatchState_WatchSuccess) - - for _, channel := range []string{"channel-1", "channel-2", "channel-3", "channel-4"} { - if channel == channelBalanced { - assert.True(t, chManager.Match(2, channel)) - } else if channel == channelBalanced2 { - assert.True(t, chManager.Match(3, channel)) - } else { - assert.True(t, chManager.Match(1, channel)) - } - } - - chManager.DeleteNode(3) - key = path.Join(prefix, "2", channelBalanced2) - waitAndStore(t, watchkv, key, datapb.ChannelWatchState_ToWatch, datapb.ChannelWatchState_WatchSuccess) - - for _, channel := range []string{"channel-1", "channel-2", "channel-3", "channel-4"} { - if channel == channelBalanced { - assert.True(t, chManager.Match(2, channel)) - } else if channel == channelBalanced2 { - assert.True(t, chManager.Match(2, channel)) - } else { - assert.True(t, chManager.Match(1, channel)) - } - } - - chManager.DeleteNode(2) - key = path.Join(prefix, "1", channelBalanced) - waitAndStore(t, watchkv, key, datapb.ChannelWatchState_ToWatch, datapb.ChannelWatchState_WatchSuccess) - key = path.Join(prefix, "1", channelBalanced2) - waitAndStore(t, watchkv, key, datapb.ChannelWatchState_ToWatch, datapb.ChannelWatchState_WatchSuccess) - - for _, channel := range []string{"channel-1", "channel-2", "channel-3", "channel-4"} { - assert.True(t, chManager.Match(1, channel)) - } - }) -} - -func TestChannelManager_RemoveChannel(t *testing.T) { - watchkv := getWatchKV(t) - defer func() { - watchkv.RemoveWithPrefix("") - watchkv.Close() - }() - - type fields struct { - store RWChannelStore - } - type args struct { - channelName string - } - tests := []struct { - name string - fields fields - args args - wantErr bool - }{ - { - "test remove existed channel", - fields{ - store: &ChannelStore{ - store: watchkv, - channelsInfo: map[int64]*NodeChannelInfo{ - 1: NewNodeChannelInfo(1, &channelMeta{Name: "ch1", CollectionID: 1}), - }, - }, - }, - args{ - "ch1", - }, - false, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - c := &ChannelManagerImpl{ - store: tt.fields.store, - } - err := c.RemoveChannel(tt.args.channelName) - assert.Equal(t, tt.wantErr, err != nil) - _, ch := c.findChannel(tt.args.channelName) - assert.Nil(t, ch) - }) - } -} - -func TestChannelManager_HelperFunc(t *testing.T) { - c := &ChannelManagerImpl{} - t.Run("test getOldOnlines", func(t *testing.T) { - tests := []struct { - nodes []int64 - oNodes []int64 - - expectedOut []int64 - desription string - }{ - {[]int64{}, []int64{}, []int64{}, "empty both"}, - {[]int64{1}, []int64{}, []int64{}, "empty oNodes"}, - {[]int64{}, []int64{1}, []int64{}, "empty nodes"}, - {[]int64{1}, []int64{1}, []int64{1}, "same one"}, - {[]int64{1, 2}, []int64{1}, []int64{1}, "same one 2"}, - {[]int64{1}, []int64{1, 2}, []int64{1}, "same one 3"}, - {[]int64{1, 2}, []int64{1, 2}, []int64{1, 2}, "same two"}, - } - - for _, test := range tests { - t.Run(test.desription, func(t *testing.T) { - nodes := c.getOldOnlines(test.nodes, test.oNodes) - assert.ElementsMatch(t, test.expectedOut, nodes) - }) - } - }) - - t.Run("test getNewOnLines", func(t *testing.T) { - tests := []struct { - nodes []int64 - oNodes []int64 - - expectedOut []int64 - desription string - }{ - {[]int64{}, []int64{}, []int64{}, "empty both"}, - {[]int64{1}, []int64{}, []int64{1}, "empty oNodes"}, - {[]int64{}, []int64{1}, []int64{}, "empty nodes"}, - {[]int64{1}, []int64{1}, []int64{}, "same one"}, - {[]int64{1, 2}, []int64{1}, []int64{2}, "same one 2"}, - {[]int64{1}, []int64{1, 2}, []int64{}, "same one 3"}, - {[]int64{1, 2}, []int64{1, 2}, []int64{}, "same two"}, - } - - for _, test := range tests { - t.Run(test.desription, func(t *testing.T) { - nodes := c.getNewOnLines(test.nodes, test.oNodes) - assert.ElementsMatch(t, test.expectedOut, nodes) - }) - } - }) -} - -func TestChannelManager_BackgroundChannelChecker(t *testing.T) { - Params.Save(Params.DataCoordCfg.AutoBalance.Key, "false") - Params.Save(Params.DataCoordCfg.ChannelBalanceInterval.Key, "1") - Params.Save(Params.DataCoordCfg.ChannelBalanceSilentDuration.Key, "1") - - watchkv := getWatchKV(t) - defer func() { - watchkv.RemoveWithPrefix("") - watchkv.Close() - }() - - defer watchkv.RemoveWithPrefix("") - - c, err := NewChannelManager(watchkv, newMockHandler(), withStateChecker()) - require.NoError(t, err) - mockStore := NewMockRWChannelStore(t) - mockStore.EXPECT().GetNodesChannels().Return([]*NodeChannelInfo{ - { - NodeID: 1, - Channels: map[string]RWChannel{ - "channel-1": &channelMeta{ - Name: "channel-1", - }, - "channel-2": &channelMeta{ - Name: "channel-2", - }, - "channel-3": &channelMeta{ - Name: "channel-3", - }, - }, - }, - { - NodeID: 2, - }, - }).Maybe() - c.store = mockStore - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - go c.bgCheckChannelsWork(ctx) - - updateCounter := atomic.NewInt64(0) - mockStore.EXPECT().Update(mock.Anything).Run(func(op *ChannelOpSet) { - updateCounter.Inc() - }).Return(nil).Maybe() - - t.Run("test disable auto balance", func(t *testing.T) { - assert.Eventually(t, func() bool { - return updateCounter.Load() == 0 - }, 5*time.Second, 1*time.Second) - }) - - t.Run("test enable auto balance", func(t *testing.T) { - Params.Save(Params.DataCoordCfg.AutoBalance.Key, "true") - assert.Eventually(t, func() bool { - return updateCounter.Load() > 0 - }, 5*time.Second, 1*time.Second) - }) -} diff --git a/internal/datacoord/channel_store.go b/internal/datacoord/channel_store.go index a8b2bb7fc8d95..ab30a0b23083f 100644 --- a/internal/datacoord/channel_store.go +++ b/internal/datacoord/channel_store.go @@ -32,7 +32,6 @@ import ( "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" - "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/timerecord" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -154,9 +153,7 @@ func (op *ChannelOp) BuildKV() (map[string]string, []string, error) { switch op.Type { case Add, Watch, Release: tmpWatchInfo := proto.Clone(ch.GetWatchInfo()).(*datapb.ChannelWatchInfo) - if paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.GetAsBool() { - tmpWatchInfo.Vchan = reduceVChanSize(tmpWatchInfo.GetVchan()) - } + tmpWatchInfo.Vchan = reduceVChanSize(tmpWatchInfo.GetVchan()) info, err := proto.Marshal(tmpWatchInfo) if err != nil { return saves, removals, err diff --git a/internal/datacoord/channel_store_test.go b/internal/datacoord/channel_store_test.go index 0790fc6a2ea15..2f82b0aaec7f7 100644 --- a/internal/datacoord/channel_store_test.go +++ b/internal/datacoord/channel_store_test.go @@ -22,16 +22,12 @@ import ( "testing" "github.com/golang/protobuf/proto" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" - "github.com/milvus-io/milvus/internal/kv" "github.com/milvus-io/milvus/internal/kv/mocks" - "github.com/milvus-io/milvus/internal/kv/predicates" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/metrics" - "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/testutils" ) @@ -58,55 +54,6 @@ func genChannelOperationsV1(from, to int64, num int) *ChannelOpSet { return ops } -func TestChannelStore_Update(t *testing.T) { - enableRPCK := paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.Key - paramtable.Get().Save(enableRPCK, "false") - defer paramtable.Get().Reset(enableRPCK) - txnKv := mocks.NewTxnKV(t) - txnKv.EXPECT().MultiSaveAndRemove(mock.Anything, mock.Anything).Run(func(saves map[string]string, removals []string, preds ...predicates.Predicate) { - assert.False(t, len(saves)+len(removals) > 64, "too many operations") - }).Return(nil) - - type fields struct { - store kv.TxnKV - channelsInfo map[int64]*NodeChannelInfo - } - type args struct { - opSet *ChannelOpSet - } - tests := []struct { - name string - fields fields - args args - wantErr bool - }{ - { - "test more than 128 operations", - fields{ - txnKv, - map[int64]*NodeChannelInfo{ - 1: genNodeChannelInfos(1, 500), - 2: NewNodeChannelInfo(2), - }, - }, - args{ - genChannelOperationsV1(1, 2, 250), - }, - false, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - c := &ChannelStore{ - store: tt.fields.store, - channelsInfo: tt.fields.channelsInfo, - } - err := c.Update(tt.args.opSet) - assert.Equal(t, tt.wantErr, err != nil) - }) - } -} - type ChannelStoreReloadSuite struct { testutils.PromMetricsSuite diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index 9eb87923f8976..77c8b9ca3c7d8 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -487,16 +487,9 @@ func (s *Server) initCluster() error { s.sessionManager = NewSessionManagerImpl(withSessionCreator(s.dataNodeCreator)) var err error - if paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.GetAsBool() { - s.channelManager, err = NewChannelManagerV2(s.watchClient, s.handler, s.sessionManager, s.allocator, withCheckerV2()) - if err != nil { - return err - } - } else { - s.channelManager, err = NewChannelManager(s.watchClient, s.handler, withMsgstreamFactory(s.factory), withStateChecker(), withBgChecker()) - if err != nil { - return err - } + s.channelManager, err = NewChannelManagerV2(s.watchClient, s.handler, s.sessionManager, s.allocator, withCheckerV2()) + if err != nil { + return err } s.cluster = NewClusterImpl(s.sessionManager, s.channelManager) return nil diff --git a/internal/datanode/channel_manager.go b/internal/datanode/channel/channel_manager.go similarity index 81% rename from internal/datanode/channel_manager.go rename to internal/datanode/channel/channel_manager.go index 1fb3e4d4a01eb..2c300b8edf0a3 100644 --- a/internal/datanode/channel_manager.go +++ b/internal/datanode/channel/channel_manager.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package channel import ( "context" @@ -22,19 +22,21 @@ import ( "time" "github.com/cockroachdb/errors" - "go.uber.org/atomic" "go.uber.org/zap" + "github.com/milvus-io/milvus/internal/datanode/pipeline" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/lifetime" "github.com/milvus-io/milvus/pkg/util/merr" + "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) type ( releaseFunc func(channel string) - watchFunc func(ctx context.Context, dn *DataNode, info *datapb.ChannelWatchInfo, tickler *tickler) (*dataSyncService, error) + watchFunc func(ctx context.Context, pipelineParams *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler) (*pipeline.DataSyncService, error) ) type ChannelManager interface { @@ -45,10 +47,10 @@ type ChannelManager interface { } type ChannelManagerImpl struct { - mu sync.RWMutex - dn *DataNode + mu sync.RWMutex + pipelineParams *util.PipelineParams - fgManager FlowgraphManager + fgManager pipeline.FlowgraphManager communicateCh chan *opState opRunners *typeutil.ConcurrentMap[string, *opRunner] // channel -> runner @@ -60,16 +62,16 @@ type ChannelManagerImpl struct { closeWaiter sync.WaitGroup } -func NewChannelManager(dn *DataNode) *ChannelManagerImpl { +func NewChannelManager(pipelineParams *util.PipelineParams, fgManager pipeline.FlowgraphManager) *ChannelManagerImpl { cm := ChannelManagerImpl{ - dn: dn, - fgManager: dn.flowgraphManager, + pipelineParams: pipelineParams, + fgManager: nil, communicateCh: make(chan *opState, 100), opRunners: typeutil.NewConcurrentMap[string, *opRunner](), abnormals: typeutil.NewConcurrentMap[int64, string](), - releaseFunc: dn.flowgraphManager.RemoveFlowgraph, + releaseFunc: fgManager.RemoveFlowgraph, closeCh: lifetime.NewSafeChan(), } @@ -209,7 +211,7 @@ func (m *ChannelManagerImpl) handleOpState(opState *opState) { } func (m *ChannelManagerImpl) getOrCreateRunner(channel string) *opRunner { - runner, loaded := m.opRunners.GetOrInsert(channel, NewOpRunner(channel, m.dn, m.releaseFunc, executeWatch, m.communicateCh)) + runner, loaded := m.opRunners.GetOrInsert(channel, NewOpRunner(channel, m.pipelineParams, m.releaseFunc, executeWatch, m.communicateCh)) if !loaded { runner.Start() } @@ -224,17 +226,17 @@ func (m *ChannelManagerImpl) finishOp(opID int64, channel string) { } type opInfo struct { - tickler *tickler + tickler *util.Tickler } type opRunner struct { - channel string - dn *DataNode - releaseFunc releaseFunc - watchFunc watchFunc + channel string + pipelineParams *util.PipelineParams + releaseFunc releaseFunc + watchFunc watchFunc guard sync.RWMutex - allOps map[UniqueID]*opInfo // opID -> tickler + allOps map[util.UniqueID]*opInfo // opID -> tickler opsInQueue chan *datapb.ChannelWatchInfo resultCh chan *opState @@ -242,16 +244,16 @@ type opRunner struct { closeWg sync.WaitGroup } -func NewOpRunner(channel string, dn *DataNode, releaseF releaseFunc, watchF watchFunc, resultCh chan *opState) *opRunner { +func NewOpRunner(channel string, pipelineParams *util.PipelineParams, releaseF releaseFunc, watchF watchFunc, resultCh chan *opState) *opRunner { return &opRunner{ - channel: channel, - dn: dn, - releaseFunc: releaseF, - watchFunc: watchF, - opsInQueue: make(chan *datapb.ChannelWatchInfo, 10), - allOps: make(map[UniqueID]*opInfo), - resultCh: resultCh, - closeCh: lifetime.NewSafeChan(), + channel: channel, + pipelineParams: pipelineParams, + releaseFunc: releaseF, + watchFunc: watchF, + opsInQueue: make(chan *datapb.ChannelWatchInfo, 10), + allOps: make(map[util.UniqueID]*opInfo), + resultCh: resultCh, + closeCh: lifetime.NewSafeChan(), } } @@ -270,13 +272,13 @@ func (r *opRunner) Start() { }() } -func (r *opRunner) FinishOp(opID UniqueID) { +func (r *opRunner) FinishOp(opID util.UniqueID) { r.guard.Lock() defer r.guard.Unlock() delete(r.allOps, opID) } -func (r *opRunner) Exist(opID UniqueID) bool { +func (r *opRunner) Exist(opID util.UniqueID) bool { r.guard.RLock() defer r.guard.RUnlock() _, ok := r.allOps[opID] @@ -334,7 +336,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState { opState.state = datapb.ChannelWatchState_WatchFailure return opState } - tickler := newTickler() + tickler := util.NewTickler() opInfo.tickler = tickler var ( @@ -342,7 +344,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState { finishWaiter sync.WaitGroup ) - watchTimeout := Params.DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second) + watchTimeout := paramtable.Get().DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second) ctx, cancel := context.WithTimeout(context.Background(), watchTimeout) defer cancel() @@ -358,19 +360,19 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState { select { case <-timer.C: // watch timeout - tickler.close() + tickler.Close() cancel() log.Info("Stop timer for ToWatch operation timeout") return case <-r.closeCh.CloseCh(): // runner closed from outside - tickler.close() + tickler.Close() cancel() log.Info("Suspend ToWatch operation from outside of opRunner") return - case <-tickler.progressSig: + case <-tickler.GetProgressSig(): log.Info("Reset timer for tickler updated") timer.Reset(watchTimeout) @@ -387,7 +389,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState { go func() { defer finishWaiter.Done() - fg, err := r.watchFunc(ctx, r.dn, info, tickler) + fg, err := r.watchFunc(ctx, r.pipelineParams, info, tickler) if err != nil { opState.state = datapb.ChannelWatchState_WatchFailure } else { @@ -402,7 +404,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState { } // releaseWithTimer will return ReleaseFailure after WatchTimeoutInterval -func (r *opRunner) releaseWithTimer(releaseFunc releaseFunc, channel string, opID UniqueID) *opState { +func (r *opRunner) releaseWithTimer(releaseFunc releaseFunc, channel string, opID util.UniqueID) *opState { opState := &opState{ channel: channel, opID: opID, @@ -416,7 +418,7 @@ func (r *opRunner) releaseWithTimer(releaseFunc releaseFunc, channel string, opI startTimer := func(finishWaiter *sync.WaitGroup) { defer finishWaiter.Done() - releaseTimeout := Params.DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second) + releaseTimeout := paramtable.Get().DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second) timer := time.NewTimer(releaseTimeout) defer timer.Stop() @@ -474,61 +476,17 @@ type opState struct { channel string opID int64 state datapb.ChannelWatchState - fg *dataSyncService + fg *pipeline.DataSyncService } // executeWatch will always return, won't be stuck, either success or fail. -func executeWatch(ctx context.Context, dn *DataNode, info *datapb.ChannelWatchInfo, tickler *tickler) (*dataSyncService, error) { - dataSyncService, err := newDataSyncService(ctx, dn, info, tickler) +func executeWatch(ctx context.Context, pipelineParams *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler) (*pipeline.DataSyncService, error) { + dataSyncService, err := pipeline.NewDataSyncService(ctx, pipelineParams, info, tickler) if err != nil { return nil, err } - dataSyncService.start() + dataSyncService.Start() return dataSyncService, nil } - -// tickler counts every time when called inc(), -type tickler struct { - count *atomic.Int32 - total *atomic.Int32 - closedSig *atomic.Bool - - progressSig chan struct{} -} - -func (t *tickler) inc() { - t.count.Inc() - t.progressSig <- struct{}{} -} - -func (t *tickler) setTotal(total int32) { - t.total.Store(total) -} - -// progress returns the count over total if total is set -// else just return the count number. -func (t *tickler) progress() int32 { - if t.total.Load() == 0 { - return t.count.Load() - } - return (t.count.Load() / t.total.Load()) * 100 -} - -func (t *tickler) close() { - t.closedSig.CompareAndSwap(false, true) -} - -func (t *tickler) closed() bool { - return t.closedSig.Load() -} - -func newTickler() *tickler { - return &tickler{ - count: atomic.NewInt32(0), - total: atomic.NewInt32(0), - closedSig: atomic.NewBool(false), - progressSig: make(chan struct{}, 200), - } -} diff --git a/internal/datanode/channel_manager_test.go b/internal/datanode/channel/channel_manager_test.go similarity index 67% rename from internal/datanode/channel_manager_test.go rename to internal/datanode/channel/channel_manager_test.go index 0dad91c14c786..4bbe3b27a4d2b 100644 --- a/internal/datanode/channel_manager_test.go +++ b/internal/datanode/channel/channel_manager_test.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package channel import ( "context" @@ -23,11 +23,14 @@ import ( "github.com/cockroachdb/errors" "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" "github.com/milvus-io/milvus/internal/datanode/allocator" + broker2 "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/proto/datapb" - "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/paramtable" ) @@ -41,11 +44,14 @@ func TestOpRunnerSuite(t *testing.T) { } func (s *OpRunnerSuite) SetupTest() { - ctx := context.Background() - s.mockAlloc = allocator.NewMockAllocator(s.T()) - - s.node = newIDLEDataNodeMock(ctx, schemapb.DataType_Int64) - s.node.allocator = s.mockAlloc + broker := broker2.NewMockBroker(s.T()) + s.pipelineParams = &util.PipelineParams{ + Ctx: context.TODO(), + Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 0}}, + CheckpointUpdater: util.NewChannelCheckpointUpdater(broker), + SyncMgr: syncmgr.NewMockSyncManager(s.T()), + Allocator: allocator.NewMockAllocator(s.T()), + } } func (s *OpRunnerSuite) TestWatchWithTimer() { @@ -53,11 +59,12 @@ func (s *OpRunnerSuite) TestWatchWithTimer() { channel string = "ch-1" commuCh = make(chan *opState) ) - info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) + info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) mockReleaseFunc := func(channel string) { log.Info("mock release func") } - runner := NewOpRunner(channel, s.node, mockReleaseFunc, executeWatch, commuCh) + + runner := NewOpRunner(channel, s.pipelineParams, mockReleaseFunc, executeWatch, commuCh) err := runner.Enqueue(info) s.Require().NoError(err) @@ -70,21 +77,21 @@ func (s *OpRunnerSuite) TestWatchWithTimer() { func (s *OpRunnerSuite) TestWatchTimeout() { channel := "by-dev-rootcoord-dml-1000" - paramtable.Get().Save(Params.DataCoordCfg.WatchTimeoutInterval.Key, "0.000001") - defer paramtable.Get().Reset(Params.DataCoordCfg.WatchTimeoutInterval.Key) - info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) + paramtable.Get().Save(datanode.Params.DataCoordCfg.WatchTimeoutInterval.Key, "0.000001") + defer paramtable.Get().Reset(datanode.Params.DataCoordCfg.WatchTimeoutInterval.Key) + info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) sig := make(chan struct{}) commuCh := make(chan *opState) mockReleaseFunc := func(channel string) { log.Info("mock release func") } - mockWatchFunc := func(ctx context.Context, dn *DataNode, info *datapb.ChannelWatchInfo, tickler *tickler) (*dataSyncService, error) { + mockWatchFunc := func(ctx context.Context, param *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler) (*pipeline.DataSyncService, error) { <-ctx.Done() sig <- struct{}{} return nil, errors.New("timeout") } - runner := NewOpRunner(channel, s.node, mockReleaseFunc, mockWatchFunc, commuCh) + runner := NewOpRunner(channel, s.pipelineParams, mockReleaseFunc, mockWatchFunc, commuCh) runner.Start() defer runner.Close() err := runner.Enqueue(info) @@ -99,55 +106,27 @@ func (s *OpRunnerSuite) TestWatchTimeout() { type OpRunnerSuite struct { suite.Suite - node *DataNode - mockAlloc *allocator.MockAllocator + pipelineParams *util.PipelineParams } type ChannelManagerSuite struct { suite.Suite - node *DataNode - manager *ChannelManagerImpl + pipelineParams *util.PipelineParams + manager *ChannelManagerImpl } func (s *ChannelManagerSuite) SetupTest() { - ctx := context.Background() - s.node = newIDLEDataNodeMock(ctx, schemapb.DataType_Int64) - s.node.allocator = allocator.NewMockAllocator(s.T()) - s.node.flowgraphManager = newFlowgraphManager() - - s.manager = NewChannelManager(s.node) -} - -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"}, - }, - }, - }, - }, + broker := broker2.NewMockBroker(s.T()) + s.pipelineParams = &util.PipelineParams{ + Ctx: context.TODO(), + Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 0}}, + CheckpointUpdater: util.NewChannelCheckpointUpdater(broker), + SyncMgr: syncmgr.NewMockSyncManager(s.T()), + Allocator: allocator.NewMockAllocator(s.T()), } + + s.manager = NewChannelManager(s.pipelineParams, pipeline.NewFlowgraphManager()) } func (s *ChannelManagerSuite) TearDownTest() { @@ -165,7 +144,7 @@ func (s *ChannelManagerSuite) TestReleaseStuck() { stuckSig <- struct{}{} } - info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) + info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) s.Require().Equal(0, s.manager.opRunners.Len()) err := s.manager.Submit(info) s.Require().NoError(err) @@ -175,9 +154,9 @@ func (s *ChannelManagerSuite) TestReleaseStuck() { s.manager.handleOpState(opState) - releaseInfo := getWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease) - paramtable.Get().Save(Params.DataCoordCfg.WatchTimeoutInterval.Key, "0.1") - defer paramtable.Get().Reset(Params.DataCoordCfg.WatchTimeoutInterval.Key) + releaseInfo := util.GetWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease) + paramtable.Get().Save(datanode.Params.DataCoordCfg.WatchTimeoutInterval.Key, "0.1") + defer paramtable.Get().Reset(datanode.Params.DataCoordCfg.WatchTimeoutInterval.Key) err = s.manager.Submit(releaseInfo) s.NoError(err) @@ -201,7 +180,7 @@ func (s *ChannelManagerSuite) TestReleaseStuck() { func (s *ChannelManagerSuite) TestSubmitIdempotent() { channel := "by-dev-rootcoord-dml-1" - info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) + info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) s.Require().Equal(0, s.manager.opRunners.Len()) for i := 0; i < 10; i++ { @@ -220,7 +199,7 @@ func (s *ChannelManagerSuite) TestSubmitIdempotent() { func (s *ChannelManagerSuite) TestSubmitSkip() { channel := "by-dev-rootcoord-dml-1" - info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) + info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) s.Require().Equal(0, s.manager.opRunners.Len()) err := s.manager.Submit(info) @@ -232,7 +211,7 @@ func (s *ChannelManagerSuite) TestSubmitSkip() { s.NotNil(opState) s.Equal(datapb.ChannelWatchState_WatchSuccess, opState.state) s.NotNil(opState.fg) - s.Equal(info.GetOpID(), opState.fg.opID) + s.Equal(info.GetOpID(), opState.fg.GetOpID()) s.manager.handleOpState(opState) err = s.manager.Submit(info) @@ -247,7 +226,7 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() { channel := "by-dev-rootcoord-dml-0" // watch - info := getWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) + info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) err := s.manager.Submit(info) s.NoError(err) @@ -256,7 +235,7 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() { s.NotNil(opState) s.Equal(datapb.ChannelWatchState_WatchSuccess, opState.state) s.NotNil(opState.fg) - s.Equal(info.GetOpID(), opState.fg.opID) + s.Equal(info.GetOpID(), opState.fg.GetOpID()) resp := s.manager.GetProgress(info) s.Equal(info.GetOpID(), resp.GetOpID()) @@ -272,7 +251,7 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() { s.Equal(datapb.ChannelWatchState_WatchSuccess, resp.GetState()) // release - info = getWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease) + info = util.GetWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease) err = s.manager.Submit(info) s.NoError(err) diff --git a/internal/datanode/mock_channelmanager.go b/internal/datanode/channel/mock_channelmanager.go similarity index 99% rename from internal/datanode/mock_channelmanager.go rename to internal/datanode/channel/mock_channelmanager.go index 3b71f5d2073ba..f94f890280b0c 100644 --- a/internal/datanode/mock_channelmanager.go +++ b/internal/datanode/channel/mock_channelmanager.go @@ -1,6 +1,6 @@ // Code generated by mockery v2.32.4. DO NOT EDIT. -package datanode +package channel import ( datapb "github.com/milvus-io/milvus/internal/proto/datapb" diff --git a/internal/datanode/compaction/l0_compactor.go b/internal/datanode/compaction/l0_compactor.go index 0163d443d33be..60eeeca245749 100644 --- a/internal/datanode/compaction/l0_compactor.go +++ b/internal/datanode/compaction/l0_compactor.go @@ -30,7 +30,6 @@ import ( "github.com/milvus-io/milvus/internal/datanode/allocator" "github.com/milvus-io/milvus/internal/datanode/io" "github.com/milvus-io/milvus/internal/datanode/metacache" - "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" @@ -411,7 +410,7 @@ func (t *LevelZeroCompactionTask) loadBF(ctx context.Context, targetSegments []* future := pool.Submit(func() (any, error) { _ = binlog.DecompressBinLog(storage.StatsBinlog, segment.GetCollectionID(), segment.GetPartitionID(), segment.GetSegmentID(), segment.GetField2StatslogPaths()) - pks, err := util.LoadStats(innerCtx, t.cm, + pks, err := LoadStats(innerCtx, t.cm, t.plan.GetSchema(), segment.GetSegmentID(), segment.GetField2StatslogPaths()) if err != nil { log.Warn("failed to load segment stats log", diff --git a/internal/datanode/util/load_stats.go b/internal/datanode/compaction/load_stats.go similarity index 99% rename from internal/datanode/util/load_stats.go rename to internal/datanode/compaction/load_stats.go index f0932329b86c6..60ef9d47cc8e4 100644 --- a/internal/datanode/util/load_stats.go +++ b/internal/datanode/compaction/load_stats.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package util +package compaction import ( "context" diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index 23e968db51a72..23af382d1c0e1 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -36,9 +36,12 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/datanode/allocator" "github.com/milvus-io/milvus/internal/datanode/broker" + "github.com/milvus-io/milvus/internal/datanode/channel" "github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/datanode/importv2" + "github.com/milvus-io/milvus/internal/datanode/pipeline" "github.com/milvus-io/milvus/internal/datanode/syncmgr" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/datanode/writebuffer" "github.com/milvus-io/milvus/internal/kv" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" @@ -52,7 +55,6 @@ import ( "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/expr" "github.com/milvus-io/milvus/pkg/util/logutil" - "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/retry" "github.com/milvus-io/milvus/pkg/util/typeutil" @@ -63,8 +65,6 @@ const ( ConnectEtcdMaxRetryTime = 100 ) -var getFlowGraphServiceAttempts = uint(50) - // makes sure DataNode implements types.DataNode var _ types.DataNode = (*DataNode)(nil) @@ -88,21 +88,19 @@ type DataNode struct { cancel context.CancelFunc Role string stateCode atomic.Value // commonpb.StateCode_Initializing - flowgraphManager FlowgraphManager + flowgraphManager pipeline.FlowgraphManager - eventManager *EventManager - channelManager ChannelManager + channelManager channel.ChannelManager syncMgr syncmgr.SyncManager writeBufferManager writebuffer.BufferManager importTaskMgr importv2.TaskManager importScheduler importv2.Scheduler - clearSignal chan string // vchannel name - segmentCache *Cache + segmentCache *util.Cache compactionExecutor compaction.Executor - timeTickSender *timeTickSender - channelCheckpointUpdater *channelCheckpointUpdater + timeTickSender *util.TimeTickSender + channelCheckpointUpdater *util.ChannelCheckpointUpdater etcdCli *clientv3.Client address string @@ -114,7 +112,6 @@ type DataNode struct { initOnce sync.Once startOnce sync.Once stopOnce sync.Once - stopWaiter sync.WaitGroup sessionMu sync.Mutex // to fix data race session *sessionutil.Session watchKv kv.WatchKV @@ -139,14 +136,11 @@ func NewDataNode(ctx context.Context, factory dependency.Factory) *DataNode { cancel: cancel2, Role: typeutil.DataNodeRole, - rootCoord: nil, - dataCoord: nil, - factory: factory, - segmentCache: newCache(), - compactionExecutor: compaction.NewExecutor(), - - clearSignal: make(chan string, 100), - + rootCoord: nil, + dataCoord: nil, + factory: factory, + segmentCache: util.NewCache(), + compactionExecutor: compaction.NewExecutor(), reportImportRetryTimes: 10, } node.UpdateStateCode(commonpb.StateCode_Abnormal) @@ -215,17 +209,6 @@ func (node *DataNode) initSession() error { return nil } -// initRateCollector creates and starts rateCollector in QueryNode. -func (node *DataNode) initRateCollector() error { - err := initGlobalRateCollector() - if err != nil { - return err - } - rateCol.Register(metricsinfo.InsertConsumeThroughput) - rateCol.Register(metricsinfo.DeleteConsumeThroughput) - return nil -} - func (node *DataNode) GetNodeID() int64 { if node.session != nil { return node.session.ServerID @@ -250,7 +233,7 @@ func (node *DataNode) Init() error { node.broker = broker.NewCoordBroker(node.dataCoord, serverID) - err := node.initRateCollector() + err := util.InitGlobalRateCollector() if err != nil { log.Error("DataNode server init rateCollector failed", zap.Error(err)) initError = err @@ -292,38 +275,15 @@ func (node *DataNode) Init() error { node.importTaskMgr = importv2.NewTaskManager() node.importScheduler = importv2.NewScheduler(node.importTaskMgr) - node.channelCheckpointUpdater = newChannelCheckpointUpdater(node.broker) - node.flowgraphManager = newFlowgraphManager() + node.channelCheckpointUpdater = util.NewChannelCheckpointUpdater(node.broker) + node.flowgraphManager = pipeline.NewFlowgraphManager() - if paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.GetAsBool() { - node.channelManager = NewChannelManager(node) - } else { - node.eventManager = NewEventManager() - } + node.channelManager = channel.NewChannelManager(getPipelineParams(node), node.flowgraphManager) log.Info("init datanode done", zap.String("Address", node.address)) }) return initError } -// handleChannelEvt handles event from kv watch event -func (node *DataNode) handleChannelEvt(evt *clientv3.Event) { - var e *event - switch evt.Type { - case clientv3.EventTypePut: // datacoord shall put channels needs to be watched here - e = &event{ - eventType: putEventType, - version: evt.Kv.Version, - } - - case clientv3.EventTypeDelete: - e = &event{ - eventType: deleteEventType, - version: evt.Kv.Version, - } - } - node.handleWatchInfo(e, string(evt.Kv.Key), evt.Kv.Value) -} - // tryToReleaseFlowgraph tries to release a flowgraph func (node *DataNode) tryToReleaseFlowgraph(channel string) { log.Info("try to release flowgraph", zap.String("channel", channel)) @@ -338,22 +298,6 @@ func (node *DataNode) tryToReleaseFlowgraph(channel string) { } } -// BackGroundGC runs in background to release datanode resources -// GOOSE TODO: remove background GC, using ToRelease for drop-collection after #15846 -func (node *DataNode) BackGroundGC(vChannelCh <-chan string) { - defer node.stopWaiter.Done() - log.Info("DataNode Background GC Start") - for { - select { - case vchanName := <-vChannelCh: - node.tryToReleaseFlowgraph(vchanName) - case <-node.ctx.Done(): - log.Warn("DataNode context done, exiting background GC") - return - } - } -} - // Start will update DataNode state to HEALTHY func (node *DataNode) Start() error { var startErr error @@ -365,21 +309,6 @@ func (node *DataNode) Start() error { } log.Info("start id allocator done", zap.String("role", typeutil.DataNodeRole)) - /* - rep, err := node.rootCoord.AllocTimestamp(node.ctx, &rootcoordpb.AllocTimestampRequest{ - Base: commonpbutil.NewMsgBase( - commonpbutil.WithMsgType(commonpb.MsgType_RequestTSO), - commonpbutil.WithMsgID(0), - commonpbutil.WithSourceID(node.GetNodeID()), - ), - Count: 1, - }) - if err != nil || rep.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success { - log.Warn("fail to alloc timestamp", zap.Any("rep", rep), zap.Error(err)) - startErr = errors.New("DataNode fail to alloc timestamp") - return - }*/ - connectEtcdFn := func() error { etcdKV := etcdkv.NewEtcdKV(node.etcdCli, Params.EtcdCfg.MetaRootPath.GetValue(), etcdkv.WithRequestTimeout(paramtable.Get().ServiceParam.EtcdCfg.RequestTimeout.GetAsDuration(time.Millisecond))) @@ -394,27 +323,19 @@ func (node *DataNode) Start() error { node.writeBufferManager.Start() - node.stopWaiter.Add(1) - go node.BackGroundGC(node.clearSignal) - go node.compactionExecutor.Start(node.ctx) go node.importScheduler.Start() if Params.DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() { - node.timeTickSender = newTimeTickSender(node.broker, node.session.ServerID, + node.timeTickSender = util.NewTimeTickSender(node.broker, node.session.ServerID, retry.Attempts(20), retry.Sleep(time.Millisecond*100)) - node.timeTickSender.start() + node.timeTickSender.Start() } - go node.channelCheckpointUpdater.start() + go node.channelCheckpointUpdater.Start() - if paramtable.Get().DataCoordCfg.EnableBalanceChannelWithRPC.GetAsBool() { - node.channelManager.Start() - } else { - // Start node watch node - node.startWatchChannelsAtBackground(node.ctx) - } + node.channelManager.Start() node.UpdateStateCode(commonpb.StateCode_Healthy) }) @@ -452,10 +373,6 @@ func (node *DataNode) Stop() error { node.channelManager.Close() } - if node.eventManager != nil { - node.eventManager.CloseAll() - } - if node.writeBufferManager != nil { node.writeBufferManager.Stop() } @@ -478,7 +395,7 @@ func (node *DataNode) Stop() error { } if node.channelCheckpointUpdater != nil { - node.channelCheckpointUpdater.close() + node.channelCheckpointUpdater.Close() } if node.importScheduler != nil { @@ -487,21 +404,37 @@ func (node *DataNode) Stop() error { // Delay the cancellation of ctx to ensure that the session is automatically recycled after closed the flow graph node.cancel() - node.stopWaiter.Wait() }) return nil } -// to fix data race +// SetSession to fix data race func (node *DataNode) SetSession(session *sessionutil.Session) { node.sessionMu.Lock() defer node.sessionMu.Unlock() node.session = session } -// to fix data race +// GetSession to fix data race func (node *DataNode) GetSession() *sessionutil.Session { node.sessionMu.Lock() defer node.sessionMu.Unlock() return node.session } + +func getPipelineParams(node *DataNode) *util.PipelineParams { + return &util.PipelineParams{ + Ctx: node.ctx, + Broker: node.broker, + SyncMgr: node.syncMgr, + TimeTickSender: node.timeTickSender, + CompactionExecutor: node.compactionExecutor, + MsgStreamFactory: node.factory, + DispClient: node.dispClient, + ChunkManager: node.chunkManager, + Session: node.session, + WriteBufferManager: node.writeBufferManager, + CheckpointUpdater: node.channelCheckpointUpdater, + Allocator: node.allocator, + } +} diff --git a/internal/datanode/data_node_test.go b/internal/datanode/data_node_test.go index f4007d5540c0f..4d522623a82e2 100644 --- a/internal/datanode/data_node_test.go +++ b/internal/datanode/data_node_test.go @@ -32,21 +32,23 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datanode/broker" + "github.com/milvus-io/milvus/internal/datanode/pipeline" + "github.com/milvus-io/milvus/internal/datanode/syncmgr" + "github.com/milvus-io/milvus/internal/datanode/util" + "github.com/milvus-io/milvus/internal/datanode/writebuffer" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/sessionutil" - "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) -const returnError = "ReturnError" - -type ctxKey struct{} - func TestMain(t *testing.M) { rand.Seed(time.Now().Unix()) // init embed etcd @@ -70,7 +72,7 @@ func TestMain(t *testing.M) { paramtable.Get().Save(Params.EtcdCfg.Endpoints.Key, strings.Join(addrs, ",")) paramtable.Get().Save(Params.CommonCfg.DataCoordTimeTick.Key, Params.CommonCfg.DataCoordTimeTick.GetValue()+strconv.Itoa(rand.Int())) - rateCol, err = newRateCollector() + err = util.InitGlobalRateCollector() if err != nil { panic("init test failed, err = " + err.Error()) } @@ -79,11 +81,31 @@ func TestMain(t *testing.M) { os.Exit(code) } +func NewIDLEDataNodeMock(ctx context.Context, pkType schemapb.DataType) *DataNode { + factory := dependency.NewDefaultFactory(true) + node := NewDataNode(ctx, factory) + node.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}}) + node.dispClient = msgdispatcher.NewClient(factory, typeutil.DataNodeRole, paramtable.GetNodeID()) + + broker := &broker.MockBroker{} + broker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(nil).Maybe() + broker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).Return([]*datapb.SegmentInfo{}, nil).Maybe() + + node.broker = broker + node.timeTickSender = util.NewTimeTickSender(broker, 0) + + syncMgr, _ := syncmgr.NewSyncManager(node.chunkManager, node.allocator) + + node.syncMgr = syncMgr + node.writeBufferManager = writebuffer.NewManager(syncMgr) + + return node +} + func TestDataNode(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - - node := newIDLEDataNodeMock(ctx, schemapb.DataType_Int64) + node := NewIDLEDataNodeMock(ctx, schemapb.DataType_Int64) etcdCli, err := etcd.GetEtcdClient( Params.EtcdCfg.UseEmbedEtcd.GetAsBool(), Params.EtcdCfg.EtcdUseSSL.GetAsBool(), @@ -123,7 +145,7 @@ func TestDataNode(t *testing.T) { description string }{ {nil, false, "nil input"}, - {&RootCoordFactory{}, true, "valid input"}, + {&util.RootCoordFactory{}, true, "valid input"}, } for _, test := range tests { @@ -146,7 +168,7 @@ func TestDataNode(t *testing.T) { description string }{ {nil, false, "nil input"}, - {&DataCoordFactory{}, true, "valid input"}, + {&util.DataCoordFactory{}, true, "valid input"}, } for _, test := range tests { @@ -164,7 +186,7 @@ func TestDataNode(t *testing.T) { t.Run("Test getSystemInfoMetrics", func(t *testing.T) { emptyNode := &DataNode{} emptyNode.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}}) - emptyNode.flowgraphManager = newFlowgraphManager() + emptyNode.flowgraphManager = pipeline.NewFlowgraphManager() req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics) assert.NoError(t, err) @@ -179,64 +201,14 @@ func TestDataNode(t *testing.T) { t.Run("Test getSystemInfoMetrics with quotaMetric error", func(t *testing.T) { emptyNode := &DataNode{} emptyNode.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}}) - emptyNode.flowgraphManager = newFlowgraphManager() + emptyNode.flowgraphManager = pipeline.NewFlowgraphManager() req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics) assert.NoError(t, err) - rateCol.Deregister(metricsinfo.InsertConsumeThroughput) + util.DeregisterRateCollector(metricsinfo.InsertConsumeThroughput) resp, err := emptyNode.getSystemInfoMetrics(context.TODO(), req) assert.NoError(t, err) assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) - rateCol.Register(metricsinfo.InsertConsumeThroughput) - }) - - t.Run("Test BackGroundGC", func(t *testing.T) { - vchanNameCh := make(chan string) - node.clearSignal = vchanNameCh - node.stopWaiter.Add(1) - go node.BackGroundGC(vchanNameCh) - - testDataSyncs := []struct { - dmChannelName string - }{ - {"fake-by-dev-rootcoord-dml-backgroundgc-1"}, - {"fake-by-dev-rootcoord-dml-backgroundgc-2"}, - } - - for _, test := range testDataSyncs { - err = node.flowgraphManager.AddandStartWithEtcdTickler(node, &datapb.VchannelInfo{ - CollectionID: 1, ChannelName: test.dmChannelName, - }, &schemapb.CollectionSchema{ - Name: "test_collection", - Fields: []*schemapb.FieldSchema{ - { - FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64, - }, - { - FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64, - }, - { - FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true, - }, - { - FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector, - TypeParams: []*commonpb.KeyValuePair{ - {Key: common.DimKey, Value: "128"}, - }, - }, - }, - }, genTestTickler()) - assert.NoError(t, err) - vchanNameCh <- test.dmChannelName - } - - assert.Eventually(t, func() bool { - for _, test := range testDataSyncs { - if node.flowgraphManager.HasFlowgraph(test.dmChannelName) { - return false - } - } - return true - }, 2*time.Second, 10*time.Millisecond) + util.RegisterRateCollector(metricsinfo.InsertConsumeThroughput) }) } diff --git a/internal/datanode/event_manager.go b/internal/datanode/event_manager.go deleted file mode 100644 index 25fc282a90869..0000000000000 --- a/internal/datanode/event_manager.go +++ /dev/null @@ -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/kv" - "github.com/milvus-io/milvus/internal/proto/datapb" - "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() - } -} diff --git a/internal/datanode/event_manager_test.go b/internal/datanode/event_manager_test.go deleted file mode 100644 index 1854c8e8c8e46..0000000000000 --- a/internal/datanode/event_manager_test.go +++ /dev/null @@ -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) -} diff --git a/internal/datanode/flow_graph_manager_test.go b/internal/datanode/flow_graph_manager_test.go deleted file mode 100644 index 6d9158fe42f3c..0000000000000 --- a/internal/datanode/flow_graph_manager_test.go +++ /dev/null @@ -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) - }) -} diff --git a/internal/datanode/flush_task_counter.go b/internal/datanode/flush_task_counter.go deleted file mode 100644 index f259887af20a6..0000000000000 --- a/internal/datanode/flush_task_counter.go +++ /dev/null @@ -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 -} diff --git a/internal/datanode/flush_task_counter_test.go b/internal/datanode/flush_task_counter_test.go deleted file mode 100644 index 34956d22e1a7b..0000000000000 --- a/internal/datanode/flush_task_counter_test.go +++ /dev/null @@ -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)) -} diff --git a/internal/datanode/metrics_info.go b/internal/datanode/metrics_info.go index 407b52d625174..f7a6bc2219a19 100644 --- a/internal/datanode/metrics_info.go +++ b/internal/datanode/metrics_info.go @@ -20,6 +20,7 @@ import ( "context" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/pkg/util/hardware" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" @@ -33,7 +34,7 @@ func (node *DataNode) getQuotaMetrics() (*metricsinfo.DataNodeQuotaMetrics, erro var err error rms := make([]metricsinfo.RateMetric, 0) getRateMetric := func(label metricsinfo.RateMetricLabel) { - rate, err2 := rateCol.Rate(label, ratelimitutil.DefaultAvgDuration) + rate, err2 := util.RateCol.Rate(label, ratelimitutil.DefaultAvgDuration) if err2 != nil { err = err2 return @@ -49,7 +50,7 @@ func (node *DataNode) getQuotaMetrics() (*metricsinfo.DataNodeQuotaMetrics, erro return nil, err } - minFGChannel, minFGTt := rateCol.getMinFlowGraphTt() + minFGChannel, minFGTt := util.RateCol.GetMinFlowGraphTt() return &metricsinfo.DataNodeQuotaMetrics{ Hms: metricsinfo.HardwareMetrics{}, Rms: rms, diff --git a/internal/datanode/data_sync_service.go b/internal/datanode/pipeline/data_sync_service.go similarity index 54% rename from internal/datanode/data_sync_service.go rename to internal/datanode/pipeline/data_sync_service.go index 10c2534f17db2..3ab4b4923c154 100644 --- a/internal/datanode/data_sync_service.go +++ b/internal/datanode/pipeline/data_sync_service.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package pipeline import ( "context" @@ -23,7 +23,6 @@ import ( "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/datanode/allocator" "github.com/milvus-io/milvus/internal/datanode/broker" "github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/datanode/io" @@ -45,33 +44,27 @@ import ( "github.com/milvus-io/milvus/pkg/util/typeutil" ) -// dataSyncService controls a flowgraph for a specific collection -type dataSyncService struct { +// DataSyncService controls a flowgraph for a specific collection +type DataSyncService struct { ctx context.Context cancelFn context.CancelFunc metacache metacache.MetaCache opID int64 - collectionID UniqueID // collection id of vchan for which this data sync service serves + collectionID util.UniqueID // collection id of vchan for which this data sync service serves vchannelName string // TODO: should be equal to paramtable.GetNodeID(), but intergrationtest has 1 paramtable for a minicluster, the NodeID - // varies, will cause savebinglogpath check fail. So we pass ServerID into dataSyncService to aviod it failure. - serverID UniqueID + // varies, will cause savebinglogpath check fail. So we pass ServerID into DataSyncService to aviod it failure. + serverID util.UniqueID fg *flowgraph.TimeTickedFlowGraph // internal flowgraph processes insert/delta messages broker broker.Broker syncMgr syncmgr.SyncManager - flushCh chan flushMsg - resendTTCh chan resendTTMsg // chan to ask for resending DataNode time tick message. - timetickSender *timeTickSender // reference to timeTickSender - compactor compaction.Executor // reference to compaction executor - flushingSegCache *Cache // a guarding cache stores currently flushing segment ids + timetickSender *util.TimeTickSender // reference to TimeTickSender + compactor compaction.Executor // reference to compaction executor - clearSignal chan<- string // signal channel to notify flowgraph close for collection/partition drop msg consumed - idAllocator allocator.Allocator // id/timestamp allocator - msFactory msgstream.Factory dispClient msgdispatcher.Client chunkManager storage.ChunkManager @@ -80,15 +73,14 @@ type dataSyncService struct { type nodeConfig struct { msFactory msgstream.Factory // msgStream factory - collectionID UniqueID + collectionID util.UniqueID vChannelName string metacache metacache.MetaCache - allocator allocator.Allocator - serverID UniqueID + serverID util.UniqueID } -// start the flow graph in dataSyncService -func (dsService *dataSyncService) start() { +// Start the flow graph in dataSyncService +func (dsService *DataSyncService) Start() { if dsService.fg != nil { log.Info("dataSyncService starting flow graph", zap.Int64("collectionID", dsService.collectionID), zap.String("vChanName", dsService.vchannelName)) @@ -99,7 +91,7 @@ func (dsService *dataSyncService) start() { } } -func (dsService *dataSyncService) GracefullyClose() { +func (dsService *DataSyncService) GracefullyClose() { if dsService.fg != nil { log.Info("dataSyncService gracefully closing flowgraph") dsService.fg.SetCloseMethod(flowgraph.CloseGracefully) @@ -107,7 +99,11 @@ func (dsService *dataSyncService) GracefullyClose() { } } -func (dsService *dataSyncService) close() { +func (dsService *DataSyncService) GetOpID() int64 { + return dsService.opID +} + +func (dsService *DataSyncService) close() { dsService.stopOnce.Do(func() { log := log.Ctx(dsService.ctx).With( zap.Int64("collectionID", dsService.collectionID), @@ -130,19 +126,16 @@ func (dsService *dataSyncService) close() { }) } -func getMetaCacheWithTickler(initCtx context.Context, node *DataNode, info *datapb.ChannelWatchInfo, tickler *tickler, unflushed, flushed []*datapb.SegmentInfo, storageV2Cache *metacache.StorageV2Cache) (metacache.MetaCache, error) { - tickler.setTotal(int32(len(unflushed) + len(flushed))) - return initMetaCache(initCtx, storageV2Cache, node.chunkManager, info, tickler, unflushed, flushed) +func (dsService *DataSyncService) GetMetaCache() metacache.MetaCache { + return dsService.metacache } -func getMetaCacheWithEtcdTickler(initCtx context.Context, node *DataNode, info *datapb.ChannelWatchInfo, tickler *etcdTickler, unflushed, flushed []*datapb.SegmentInfo, storageV2Cache *metacache.StorageV2Cache) (metacache.MetaCache, error) { - tickler.watch() - defer tickler.stop() - - return initMetaCache(initCtx, storageV2Cache, node.chunkManager, info, tickler, unflushed, flushed) +func getMetaCacheWithTickler(initCtx context.Context, params *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler, unflushed, flushed []*datapb.SegmentInfo, storageV2Cache *metacache.StorageV2Cache) (metacache.MetaCache, error) { + tickler.SetTotal(int32(len(unflushed) + len(flushed))) + return initMetaCache(initCtx, storageV2Cache, params.ChunkManager, info, tickler, unflushed, flushed) } -func initMetaCache(initCtx context.Context, storageV2Cache *metacache.StorageV2Cache, chunkManager storage.ChunkManager, info *datapb.ChannelWatchInfo, tickler interface{ inc() }, unflushed, flushed []*datapb.SegmentInfo) (metacache.MetaCache, error) { +func initMetaCache(initCtx context.Context, storageV2Cache *metacache.StorageV2Cache, chunkManager storage.ChunkManager, info *datapb.ChannelWatchInfo, tickler interface{ Inc() }, unflushed, flushed []*datapb.SegmentInfo) (metacache.MetaCache, error) { // tickler will update addSegment progress to watchInfo futures := make([]*conc.Future[any], 0, len(unflushed)+len(flushed)) segmentPks := typeutil.NewConcurrentMap[int64, []*storage.PkStatistics]() @@ -161,15 +154,15 @@ func initMetaCache(initCtx context.Context, storageV2Cache *metacache.StorageV2C var stats []*storage.PkStatistics var err error if params.Params.CommonCfg.EnableStorageV2.GetAsBool() { - stats, err = util.LoadStatsV2(storageV2Cache, segment, info.GetSchema()) + stats, err = compaction.LoadStatsV2(storageV2Cache, segment, info.GetSchema()) } else { - stats, err = util.LoadStats(initCtx, chunkManager, info.GetSchema(), segment.GetID(), segment.GetStatslogs()) + stats, err = compaction.LoadStats(initCtx, chunkManager, info.GetSchema(), segment.GetID(), segment.GetStatslogs()) } if err != nil { return nil, err } segmentPks.Insert(segment.GetID(), stats) - tickler.inc() + tickler.Inc() return struct{}{}, nil }) @@ -198,88 +191,77 @@ func initMetaCache(initCtx context.Context, storageV2Cache *metacache.StorageV2C return metacache, nil } -func getServiceWithChannel(initCtx context.Context, node *DataNode, info *datapb.ChannelWatchInfo, metacache metacache.MetaCache, storageV2Cache *metacache.StorageV2Cache, unflushed, flushed []*datapb.SegmentInfo) (*dataSyncService, error) { +func getServiceWithChannel(initCtx context.Context, params *util.PipelineParams, info *datapb.ChannelWatchInfo, metacache metacache.MetaCache, storageV2Cache *metacache.StorageV2Cache, unflushed, flushed []*datapb.SegmentInfo) (*DataSyncService, error) { var ( channelName = info.GetVchan().GetChannelName() collectionID = info.GetVchan().GetCollectionID() ) config := &nodeConfig{ - msFactory: node.factory, - allocator: node.allocator, - + msFactory: params.MsgStreamFactory, collectionID: collectionID, vChannelName: channelName, metacache: metacache, - serverID: node.session.ServerID, + serverID: params.Session.ServerID, } - var ( - flushCh = make(chan flushMsg, 100) - resendTTCh = make(chan resendTTMsg, 100) - ) - - err := node.writeBufferManager.Register(channelName, metacache, storageV2Cache, writebuffer.WithMetaWriter(syncmgr.BrokerMetaWriter(node.broker, config.serverID)), writebuffer.WithIDAllocator(node.allocator)) + err := params.WriteBufferManager.Register(channelName, metacache, storageV2Cache, + writebuffer.WithMetaWriter(syncmgr.BrokerMetaWriter(params.Broker, config.serverID)), + writebuffer.WithIDAllocator(params.Allocator)) if err != nil { log.Warn("failed to register channel buffer", zap.Error(err)) return nil, err } defer func() { if err != nil { - defer node.writeBufferManager.RemoveChannel(channelName) + defer params.WriteBufferManager.RemoveChannel(channelName) } }() - ctx, cancel := context.WithCancel(node.ctx) - ds := &dataSyncService{ - ctx: ctx, - cancelFn: cancel, - flushCh: flushCh, - resendTTCh: resendTTCh, - opID: info.GetOpID(), + ctx, cancel := context.WithCancel(params.Ctx) + ds := &DataSyncService{ + ctx: ctx, + cancelFn: cancel, + opID: info.GetOpID(), - dispClient: node.dispClient, - msFactory: node.factory, - broker: node.broker, + dispClient: params.DispClient, + broker: params.Broker, - idAllocator: config.allocator, metacache: config.metacache, collectionID: config.collectionID, vchannelName: config.vChannelName, serverID: config.serverID, - flushingSegCache: node.segmentCache, - clearSignal: node.clearSignal, - chunkManager: node.chunkManager, - compactor: node.compactionExecutor, - timetickSender: node.timeTickSender, - syncMgr: node.syncMgr, + chunkManager: params.ChunkManager, + compactor: params.CompactionExecutor, + timetickSender: params.TimeTickSender, + syncMgr: params.SyncMgr, fg: nil, } // init flowgraph - fg := flowgraph.NewTimeTickedFlowGraph(node.ctx) - dmStreamNode, err := newDmInputNode(initCtx, node.dispClient, info.GetVchan().GetSeekPosition(), config) + fg := flowgraph.NewTimeTickedFlowGraph(params.Ctx) + dmStreamNode, err := newDmInputNode(initCtx, params.DispClient, info.GetVchan().GetSeekPosition(), config) if err != nil { return nil, err } ddNode, err := newDDNode( - node.ctx, + params.Ctx, collectionID, channelName, info.GetVchan().GetDroppedSegmentIds(), flushed, unflushed, - node.compactionExecutor, + params.CompactionExecutor, ) if err != nil { return nil, err } var updater statsUpdater - if Params.DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() { + if paramtable.Get().DataNodeCfg.DataNodeTimeTickByRPC.GetAsBool() { updater = ds.timetickSender } else { m, err := config.msFactory.NewMsgStream(ctx) @@ -287,18 +269,18 @@ func getServiceWithChannel(initCtx context.Context, node *DataNode, info *datapb return nil, err } - m.AsProducer([]string{Params.CommonCfg.DataCoordTimeTick.GetValue()}) + m.AsProducer([]string{paramtable.Get().CommonCfg.DataCoordTimeTick.GetValue()}) metrics.DataNodeNumProducers.WithLabelValues(fmt.Sprint(config.serverID)).Inc() - log.Info("datanode AsProducer", zap.String("TimeTickChannelName", Params.CommonCfg.DataCoordTimeTick.GetValue())) + log.Info("datanode AsProducer", zap.String("TimeTickChannelName", paramtable.Get().CommonCfg.DataCoordTimeTick.GetValue())) m.EnableProduce(true) updater = newMqStatsUpdater(config, m) } - writeNode := newWriteNode(node.ctx, node.writeBufferManager, updater, config) + writeNode := newWriteNode(params.Ctx, params.WriteBufferManager, updater, config) - ttNode, err := newTTNode(config, node.writeBufferManager, node.channelCheckpointUpdater) + ttNode, err := newTTNode(config, params.WriteBufferManager, params.CheckpointUpdater) if err != nil { return nil, err } @@ -311,16 +293,16 @@ func getServiceWithChannel(initCtx context.Context, node *DataNode, info *datapb return ds, nil } -// newServiceWithEtcdTickler gets a dataSyncService, but flowgraphs are not running +// NewDataSyncService gets a dataSyncService, but flowgraphs are not running // initCtx is used to init the dataSyncService only, if initCtx.Canceled or initCtx.Timeout -// newServiceWithEtcdTickler stops and returns the initCtx.Err() -func newServiceWithEtcdTickler(initCtx context.Context, node *DataNode, info *datapb.ChannelWatchInfo, tickler *etcdTickler) (*dataSyncService, error) { +// NewDataSyncService stops and returns the initCtx.Err() +func NewDataSyncService(initCtx context.Context, pipelineParams *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler) (*DataSyncService, error) { // recover segment checkpoints - unflushedSegmentInfos, err := node.broker.GetSegmentInfo(initCtx, info.GetVchan().GetUnflushedSegmentIds()) + unflushedSegmentInfos, err := pipelineParams.Broker.GetSegmentInfo(initCtx, info.GetVchan().GetUnflushedSegmentIds()) if err != nil { return nil, err } - flushedSegmentInfos, err := node.broker.GetSegmentInfo(initCtx, info.GetVchan().GetFlushedSegmentIds()) + flushedSegmentInfos, err := pipelineParams.Broker.GetSegmentInfo(initCtx, info.GetVchan().GetFlushedSegmentIds()) if err != nil { return nil, err } @@ -332,42 +314,16 @@ func newServiceWithEtcdTickler(initCtx context.Context, node *DataNode, info *da return nil, err } } - // init channel meta - metaCache, err := getMetaCacheWithEtcdTickler(initCtx, node, info, tickler, unflushedSegmentInfos, flushedSegmentInfos, storageCache) - if err != nil { - return nil, err - } - - return getServiceWithChannel(initCtx, node, info, metaCache, storageCache, unflushedSegmentInfos, flushedSegmentInfos) -} - -// newDataSyncService gets a dataSyncService, but flowgraphs are not running -// initCtx is used to init the dataSyncService only, if initCtx.Canceled or initCtx.Timeout -// newDataSyncService stops and returns the initCtx.Err() -// NOTE: compactiable for event manager -func newDataSyncService(initCtx context.Context, node *DataNode, info *datapb.ChannelWatchInfo, tickler *tickler) (*dataSyncService, error) { - // recover segment checkpoints - unflushedSegmentInfos, err := node.broker.GetSegmentInfo(initCtx, info.GetVchan().GetUnflushedSegmentIds()) - if err != nil { - return nil, err - } - flushedSegmentInfos, err := node.broker.GetSegmentInfo(initCtx, info.GetVchan().GetFlushedSegmentIds()) - if err != nil { - return nil, err - } - var storageCache *metacache.StorageV2Cache - if params.Params.CommonCfg.EnableStorageV2.GetAsBool() { - storageCache, err = metacache.NewStorageV2Cache(info.Schema) - if err != nil { - return nil, err - } - } // init metaCache meta - metaCache, err := getMetaCacheWithTickler(initCtx, node, info, tickler, unflushedSegmentInfos, flushedSegmentInfos, storageCache) + metaCache, err := getMetaCacheWithTickler(initCtx, pipelineParams, info, tickler, unflushedSegmentInfos, flushedSegmentInfos, storageCache) if err != nil { return nil, err } - return getServiceWithChannel(initCtx, node, info, metaCache, storageCache, unflushedSegmentInfos, flushedSegmentInfos) + return getServiceWithChannel(initCtx, pipelineParams, info, metaCache, storageCache, unflushedSegmentInfos, flushedSegmentInfos) +} + +func NewDataSyncServiceWithMetaCache(metaCache metacache.MetaCache) *DataSyncService { + return &DataSyncService{metacache: metaCache} } diff --git a/internal/datanode/data_sync_service_test.go b/internal/datanode/pipeline/data_sync_service_test.go similarity index 67% rename from internal/datanode/data_sync_service_test.go rename to internal/datanode/pipeline/data_sync_service_test.go index 3847f3eebcbd3..0cbcb36b63cb3 100644 --- a/internal/datanode/data_sync_service_test.go +++ b/internal/datanode/pipeline/data_sync_service_test.go @@ -14,12 +14,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package pipeline import ( - "bytes" "context" - "encoding/binary" "fmt" "math" "math/rand" @@ -29,7 +27,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" - "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" @@ -37,6 +34,8 @@ import ( "github.com/milvus-io/milvus/internal/datanode/allocator" "github.com/milvus-io/milvus/internal/datanode/broker" "github.com/milvus-io/milvus/internal/datanode/metacache" + "github.com/milvus-io/milvus/internal/datanode/syncmgr" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/datanode/writebuffer" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/proto/datapb" @@ -44,7 +43,6 @@ import ( "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/common" - "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" "github.com/milvus-io/milvus/pkg/mq/msgstream" "github.com/milvus-io/milvus/pkg/util/paramtable" @@ -54,10 +52,6 @@ import ( var dataSyncServiceTestDir = "/tmp/milvus_test/data_sync_service" -func init() { - paramtable.Init() -} - func getWatchInfo(info *testInfo) *datapb.ChannelWatchInfo { return &datapb.ChannelWatchInfo{ Vchan: getVchanInfo(info), @@ -131,16 +125,16 @@ type testInfo struct { channelNil bool inMsgFactory dependency.Factory - collID UniqueID + collID util.UniqueID chanName string - ufCollID UniqueID - ufSegID UniqueID + ufCollID util.UniqueID + ufSegID util.UniqueID ufchanName string ufNor int64 - fCollID UniqueID - fSegID UniqueID + fCollID util.UniqueID + fSegID util.UniqueID fchanName string fNor int64 @@ -176,18 +170,55 @@ func TestDataSyncService_newDataSyncService(t *testing.T) { cm := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir)) defer cm.RemoveWithPrefix(ctx, cm.RootPath()) - node := newIDLEDataNodeMock(ctx, schemapb.DataType_Int64) - node.allocator = allocator.NewMockAllocator(t) + wbManager := writebuffer.NewMockBufferManager(t) + wbManager.EXPECT(). + Register(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) for _, test := range tests { t.Run(test.description, func(t *testing.T) { - node.factory = test.inMsgFactory - defer node.tryToReleaseFlowgraph(test.chanName) - ds, err := newServiceWithEtcdTickler( + mockBroker := broker.NewMockBroker(t) + mockBroker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).Call.Return( + func(_ context.Context, segmentIDs []int64) []*datapb.SegmentInfo { + data := map[int64]*datapb.SegmentInfo{ + test.fSegID: { + ID: test.fSegID, + CollectionID: test.fCollID, + PartitionID: 1, + InsertChannel: test.fchanName, + State: commonpb.SegmentState_Flushed, + }, + + test.ufSegID: { + ID: test.ufSegID, + CollectionID: test.ufCollID, + PartitionID: 1, + InsertChannel: test.ufchanName, + State: commonpb.SegmentState_Flushing, + }, + } + return lo.FilterMap(segmentIDs, func(id int64, _ int) (*datapb.SegmentInfo, bool) { + item, ok := data[id] + return item, ok + }) + }, nil) + + pipelineParams := &util.PipelineParams{ + Ctx: context.TODO(), + Broker: mockBroker, + ChunkManager: cm, + Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}}, + SyncMgr: syncmgr.NewMockSyncManager(t), + WriteBufferManager: wbManager, + Allocator: allocator.NewMockAllocator(t), + MsgStreamFactory: test.inMsgFactory, + DispClient: msgdispatcher.NewClient(test.inMsgFactory, typeutil.DataNodeRole, 1), + } + + ds, err := NewDataSyncService( ctx, - node, + pipelineParams, getWatchInfo(test), - genTestTickler(), + util.NewTickler(), ) if !test.isValidCase { @@ -199,122 +230,31 @@ func TestDataSyncService_newDataSyncService(t *testing.T) { // start ds.fg = nil - ds.start() + ds.Start() } }) } } -func TestDataSyncService_newDataSyncService_DuplicatedChannel(t *testing.T) { - ctx := context.Background() - - test := &testInfo{ - true, false, &mockMsgStreamFactory{true, true}, - 1, "by-dev-rootcoord-dml-test_v1", - 1, 1, "by-dev-rootcoord-dml-test_v1", 0, - 1, 2, "by-dev-rootcoord-dml-test_v1", 0, - "add un-flushed and flushed segments", - } - cm := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir)) - defer cm.RemoveWithPrefix(ctx, cm.RootPath()) - - watchInfo := getWatchInfo(test) - - node := newIDLEDataNodeMock(ctx, schemapb.DataType_Int64) - node.allocator = allocator.NewMockAllocator(t) - node.factory = test.inMsgFactory - metacache := metacache.NewMockMetaCache(t) - metacache.EXPECT().Collection().Return(test.collID) - metacache.EXPECT().Schema().Return(watchInfo.GetSchema()) - node.writeBufferManager.Register(test.chanName, metacache, nil, writebuffer.WithIDAllocator(allocator.NewMockAllocator(t))) - ds, err := newServiceWithEtcdTickler( - ctx, - node, - watchInfo, - genTestTickler(), - ) - - assert.Error(t, err) - assert.Nil(t, ds) -} - -func genBytes() (rawData []byte) { - const DIM = 2 - const N = 1 - - // Float vector - fvector := [DIM]float32{1, 2} - for _, ele := range fvector { - buf := make([]byte, 4) - common.Endian.PutUint32(buf, math.Float32bits(ele)) - rawData = append(rawData, buf...) - } - - // Binary vector - // Dimension of binary vector is 32 - // size := 4, = 32 / 8 - bvector := []byte{255, 255, 255, 0} - rawData = append(rawData, bvector...) - - // Bool - fieldBool := true - buf := new(bytes.Buffer) - if err := binary.Write(buf, common.Endian, fieldBool); err != nil { - panic(err) - } - - rawData = append(rawData, buf.Bytes()...) - - // int8 - var dataInt8 int8 = 100 - bint8 := new(bytes.Buffer) - if err := binary.Write(bint8, common.Endian, dataInt8); err != nil { - panic(err) - } - rawData = append(rawData, bint8.Bytes()...) - log.Debug("Rawdata length:", zap.Int("Length of rawData", len(rawData))) - return -} - -func TestBytesReader(t *testing.T) { - rawData := genBytes() - - // Bytes Reader is able to recording the position - rawDataReader := bytes.NewReader(rawData) - - fvector := make([]float32, 2) - err := binary.Read(rawDataReader, common.Endian, &fvector) - assert.NoError(t, err) - assert.ElementsMatch(t, fvector, []float32{1, 2}) - - bvector := make([]byte, 4) - err = binary.Read(rawDataReader, common.Endian, &bvector) - assert.NoError(t, err) - assert.ElementsMatch(t, bvector, []byte{255, 255, 255, 0}) - - var fieldBool bool - err = binary.Read(rawDataReader, common.Endian, &fieldBool) - assert.NoError(t, err) - assert.Equal(t, true, fieldBool) - - var dataInt8 int8 - err = binary.Read(rawDataReader, common.Endian, &dataInt8) - assert.NoError(t, err) - assert.Equal(t, int8(100), dataInt8) -} - func TestGetChannelWithTickler(t *testing.T) { channelName := "by-dev-rootcoord-dml-0" - info := getWatchInfoByOpID(100, channelName, datapb.ChannelWatchState_ToWatch) - node := newIDLEDataNodeMock(context.Background(), schemapb.DataType_Int64) - node.chunkManager = storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir)) - defer node.chunkManager.RemoveWithPrefix(context.Background(), node.chunkManager.RootPath()) - - meta := NewMetaFactory().GetCollectionMeta(1, "test_collection", schemapb.DataType_Int64) - broker := broker.NewMockBroker(t) - node.broker = broker + info := util.GetWatchInfoByOpID(100, channelName, datapb.ChannelWatchState_ToWatch) + chunkManager := storage.NewLocalChunkManager(storage.RootPath(dataSyncServiceTestDir)) + defer chunkManager.RemoveWithPrefix(context.Background(), chunkManager.RootPath()) + + meta := util.NewMetaFactory().GetCollectionMeta(1, "test_collection", schemapb.DataType_Int64) info.Schema = meta.GetSchema() + pipelineParams := &util.PipelineParams{ + Ctx: context.TODO(), + Broker: broker.NewMockBroker(t), + ChunkManager: chunkManager, + Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}}, + SyncMgr: syncmgr.NewMockSyncManager(t), + WriteBufferManager: writebuffer.NewMockBufferManager(t), + Allocator: allocator.NewMockAllocator(t), + } + unflushed := []*datapb.SegmentInfo{ { ID: 100, @@ -349,7 +289,7 @@ func TestGetChannelWithTickler(t *testing.T) { }, } - metaCache, err := getMetaCacheWithTickler(context.TODO(), node, info, newTickler(), unflushed, flushed, nil) + metaCache, err := getMetaCacheWithTickler(context.TODO(), pipelineParams, info, util.NewTickler(), unflushed, flushed, nil) assert.NoError(t, err) assert.NotNil(t, metaCache) assert.Equal(t, int64(1), metaCache.Collection()) @@ -359,49 +299,39 @@ func TestGetChannelWithTickler(t *testing.T) { type DataSyncServiceSuite struct { suite.Suite - MockDataSuiteBase - - node *DataNode // node param - chunkManager *mocks.ChunkManager - broker *broker.MockBroker - allocator *allocator.MockAllocator - wbManager *writebuffer.MockBufferManager - - factory *dependency.MockFactory - ms *msgstream.MockMsgStream - msChan chan *msgstream.MsgPack + util.MockDataSuiteBase + + pipelineParams *util.PipelineParams // node param + chunkManager *mocks.ChunkManager + broker *broker.MockBroker + allocator *allocator.MockAllocator + wbManager *writebuffer.MockBufferManager + channelCheckpointUpdater *util.ChannelCheckpointUpdater + factory *dependency.MockFactory + ms *msgstream.MockMsgStream + msChan chan *msgstream.MsgPack } func (s *DataSyncServiceSuite) SetupSuite() { paramtable.Get().Init(paramtable.NewBaseTable()) - s.MockDataSuiteBase.prepareData() + s.MockDataSuiteBase.PrepareData() } func (s *DataSyncServiceSuite) SetupTest() { - s.node = &DataNode{} - s.chunkManager = mocks.NewChunkManager(s.T()) + s.broker = broker.NewMockBroker(s.T()) + s.broker.EXPECT().UpdateSegmentStatistics(mock.Anything, mock.Anything).Return(nil).Maybe() + s.allocator = allocator.NewMockAllocator(s.T()) s.wbManager = writebuffer.NewMockBufferManager(s.T()) - s.broker.EXPECT().UpdateSegmentStatistics(mock.Anything, mock.Anything).Return(nil).Maybe() - - s.node.chunkManager = s.chunkManager - s.node.broker = s.broker - s.node.allocator = s.allocator - s.node.writeBufferManager = s.wbManager - s.node.session = &sessionutil.Session{ - SessionRaw: sessionutil.SessionRaw{ - ServerID: 1, - }, - } - s.node.ctx = context.Background() - s.node.channelCheckpointUpdater = newChannelCheckpointUpdater(s.node.broker) paramtable.Get().Save(paramtable.Get().DataNodeCfg.ChannelCheckpointUpdateTickInSeconds.Key, "0.01") defer paramtable.Get().Save(paramtable.Get().DataNodeCfg.ChannelCheckpointUpdateTickInSeconds.Key, "10") - go s.node.channelCheckpointUpdater.start() - s.msChan = make(chan *msgstream.MsgPack) + s.channelCheckpointUpdater = util.NewChannelCheckpointUpdater(s.broker) + + go s.channelCheckpointUpdater.Start() + s.msChan = make(chan *msgstream.MsgPack, 1) s.factory = dependency.NewMockFactory(s.T()) s.ms = msgstream.NewMockMsgStream(s.T()) @@ -410,18 +340,27 @@ func (s *DataSyncServiceSuite) SetupTest() { s.ms.EXPECT().Chan().Return(s.msChan) s.ms.EXPECT().Close().Return() - s.node.factory = s.factory - s.node.dispClient = msgdispatcher.NewClient(s.factory, typeutil.DataNodeRole, 1) - - s.node.timeTickSender = newTimeTickSender(s.broker, 0) + s.pipelineParams = &util.PipelineParams{ + Ctx: context.TODO(), + MsgStreamFactory: s.factory, + Broker: s.broker, + ChunkManager: s.chunkManager, + Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}}, + CheckpointUpdater: s.channelCheckpointUpdater, + SyncMgr: syncmgr.NewMockSyncManager(s.T()), + WriteBufferManager: s.wbManager, + Allocator: s.allocator, + TimeTickSender: util.NewTimeTickSender(s.broker, 0), + DispClient: msgdispatcher.NewClient(s.factory, typeutil.DataNodeRole, 1), + } } func (s *DataSyncServiceSuite) TestStartStop() { var ( insertChannelName = fmt.Sprintf("by-dev-rootcoord-dml-%d", rand.Int()) - Factory = &MetaFactory{} - collMeta = Factory.GetCollectionMeta(UniqueID(0), "coll1", schemapb.DataType_Int64) + Factory = &util.MetaFactory{} + collMeta = Factory.GetCollectionMeta(util.UniqueID(0), "coll1", schemapb.DataType_Int64) ) ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -486,30 +425,30 @@ func (s *DataSyncServiceSuite) TestStartStop() { }, } - sync, err := newServiceWithEtcdTickler( + sync, err := NewDataSyncService( ctx, - s.node, + s.pipelineParams, watchInfo, - genTestTickler(), + util.NewTickler(), ) s.Require().NoError(err) s.Require().NotNil(sync) - sync.start() + sync.Start() defer sync.close() - timeRange := TimeRange{ - timestampMin: 0, - timestampMax: math.MaxUint64 - 1, + timeRange := util.TimeRange{ + TimestampMin: 0, + TimestampMax: math.MaxUint64 - 1, } msgTs := tsoutil.GetCurrentTime() - dataFactory := NewDataFactory() + dataFactory := util.NewDataFactory() insertMessages := dataFactory.GetMsgStreamTsInsertMsgs(2, insertChannelName, msgTs) msgPack := msgstream.MsgPack{ - BeginTs: timeRange.timestampMin, - EndTs: timeRange.timestampMax, + BeginTs: timeRange.TimestampMin, + EndTs: timeRange.TimestampMax, Msgs: insertMessages, StartPositions: []*msgpb.MsgPosition{{ Timestamp: msgTs, @@ -533,7 +472,7 @@ func (s *DataSyncServiceSuite) TestStartStop() { TimeTickMsg: msgpb.TimeTickMsg{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_TimeTick, - MsgID: UniqueID(0), + MsgID: util.UniqueID(0), Timestamp: tsoutil.GetCurrentTime(), SourceID: 0, }, @@ -546,7 +485,6 @@ func (s *DataSyncServiceSuite) TestStartStop() { s.wbManager.EXPECT().NotifyCheckpointUpdated(insertChannelName, msgTs).Return().Maybe() ch := make(chan struct{}) - s.broker.EXPECT().UpdateChannelCheckpoint(mock.Anything, mock.Anything).RunAndReturn(func(_ context.Context, _ []*msgpb.MsgPosition) error { close(ch) return nil diff --git a/internal/datanode/flow_graph_dd_node.go b/internal/datanode/pipeline/flow_graph_dd_node.go similarity index 82% rename from internal/datanode/flow_graph_dd_node.go rename to internal/datanode/pipeline/flow_graph_dd_node.go index f829f1c2aeef2..5cc8acd7b8351 100644 --- a/internal/datanode/flow_graph_dd_node.go +++ b/internal/datanode/pipeline/flow_graph_dd_node.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package pipeline import ( "context" @@ -29,6 +29,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/datanode/compaction" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/pkg/log" @@ -61,15 +62,15 @@ type ddNode struct { BaseNode ctx context.Context - collectionID UniqueID + collectionID util.UniqueID vChannelName string dropMode atomic.Value compactionExecutor compaction.Executor // for recovery - growingSegInfo map[UniqueID]*datapb.SegmentInfo // segmentID - sealedSegInfo map[UniqueID]*datapb.SegmentInfo // segmentID + growingSegInfo map[util.UniqueID]*datapb.SegmentInfo // segmentID + sealedSegInfo map[util.UniqueID]*datapb.SegmentInfo // segmentID droppedSegmentIDs []int64 } @@ -99,15 +100,15 @@ func (ddn *ddNode) Operate(in []Msg) []Msg { } if msMsg.IsCloseMsg() { - fgMsg := flowGraphMsg{ + fgMsg := FlowGraphMsg{ BaseMsg: flowgraph.NewBaseMsg(true), - insertMessages: make([]*msgstream.InsertMsg, 0), - timeRange: TimeRange{ - timestampMin: msMsg.TimestampMin(), - timestampMax: msMsg.TimestampMax(), + InsertMessages: make([]*msgstream.InsertMsg, 0), + TimeRange: util.TimeRange{ + TimestampMin: msMsg.TimestampMin(), + TimestampMax: msMsg.TimestampMax(), }, - startPositions: msMsg.StartPositions(), - endPositions: msMsg.EndPositions(), + StartPositions: msMsg.StartPositions(), + EndPositions: msMsg.EndPositions(), dropCollection: false, } log.Warn("MsgStream closed", zap.Any("ddNode node", ddn.Name()), zap.String("channel", ddn.vChannelName), zap.Int64("collection", ddn.collectionID)) @@ -121,7 +122,7 @@ func (ddn *ddNode) Operate(in []Msg) []Msg { var spans []trace.Span for _, msg := range msMsg.TsMessages() { - ctx, sp := startTracer(msg, "DDNode-Operate") + ctx, sp := util.StartTracer(msg, "DDNode-Operate") spans = append(spans, sp) msg.SetTraceCtx(ctx) } @@ -131,14 +132,14 @@ func (ddn *ddNode) Operate(in []Msg) []Msg { } }() - fgMsg := flowGraphMsg{ - insertMessages: make([]*msgstream.InsertMsg, 0), - timeRange: TimeRange{ - timestampMin: msMsg.TimestampMin(), - timestampMax: msMsg.TimestampMax(), + fgMsg := FlowGraphMsg{ + InsertMessages: make([]*msgstream.InsertMsg, 0), + TimeRange: util.TimeRange{ + TimestampMin: msMsg.TimestampMin(), + TimestampMax: msMsg.TimestampMax(), }, - startPositions: make([]*msgpb.MsgPosition, 0), - endPositions: make([]*msgpb.MsgPosition, 0), + StartPositions: make([]*msgpb.MsgPosition, 0), + EndPositions: make([]*msgpb.MsgPosition, 0), dropCollection: false, } @@ -180,7 +181,7 @@ func (ddn *ddNode) Operate(in []Msg) []Msg { continue } - rateCol.Add(metricsinfo.InsertConsumeThroughput, float64(proto.Size(&imsg.InsertRequest))) + util.RateCol.Add(metricsinfo.InsertConsumeThroughput, float64(proto.Size(&imsg.InsertRequest))) metrics.DataNodeConsumeBytesCount. WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.InsertLabel). @@ -198,7 +199,7 @@ func (ddn *ddNode) Operate(in []Msg) []Msg { zap.Int64("segmentID", imsg.GetSegmentID()), zap.String("channel", ddn.vChannelName), zap.Int("numRows", len(imsg.GetRowIDs()))) - fgMsg.insertMessages = append(fgMsg.insertMessages, imsg) + fgMsg.InsertMessages = append(fgMsg.InsertMessages, imsg) case commonpb.MsgType_Delete: dmsg := msg.(*msgstream.DeleteMsg) @@ -212,7 +213,7 @@ func (ddn *ddNode) Operate(in []Msg) []Msg { } log.Debug("DDNode receive delete messages", zap.String("channel", ddn.vChannelName), zap.Int64("numRows", dmsg.NumRows)) - rateCol.Add(metricsinfo.DeleteConsumeThroughput, float64(proto.Size(&dmsg.DeleteRequest))) + util.RateCol.Add(metricsinfo.DeleteConsumeThroughput, float64(proto.Size(&dmsg.DeleteRequest))) metrics.DataNodeConsumeBytesCount. WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.DeleteLabel). @@ -225,12 +226,12 @@ func (ddn *ddNode) Operate(in []Msg) []Msg { metrics.DataNodeConsumeMsgRowsCount. WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.DeleteLabel). Add(float64(dmsg.GetNumRows())) - fgMsg.deleteMessages = append(fgMsg.deleteMessages, dmsg) + fgMsg.DeleteMessages = append(fgMsg.DeleteMessages, dmsg) } } - fgMsg.startPositions = append(fgMsg.startPositions, msMsg.StartPositions()...) - fgMsg.endPositions = append(fgMsg.endPositions, msMsg.EndPositions()...) + fgMsg.StartPositions = append(fgMsg.StartPositions, msMsg.StartPositions()...) + fgMsg.EndPositions = append(fgMsg.EndPositions, msMsg.EndPositions()...) return []Msg{&fgMsg} } @@ -267,7 +268,7 @@ func (ddn *ddNode) tryToFilterSegmentInsertMessages(msg *msgstream.InsertMsg) bo return false } -func (ddn *ddNode) isDropped(segID UniqueID) bool { +func (ddn *ddNode) isDropped(segID util.UniqueID) bool { for _, droppedSegmentID := range ddn.droppedSegmentIDs { if droppedSegmentID == segID { return true @@ -280,19 +281,19 @@ func (ddn *ddNode) Close() { log.Info("Flowgraph DD Node closing") } -func newDDNode(ctx context.Context, collID UniqueID, vChannelName string, droppedSegmentIDs []UniqueID, +func newDDNode(ctx context.Context, collID util.UniqueID, vChannelName string, droppedSegmentIDs []util.UniqueID, sealedSegments []*datapb.SegmentInfo, growingSegments []*datapb.SegmentInfo, executor compaction.Executor, ) (*ddNode, error) { baseNode := BaseNode{} - baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32()) - baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32()) + baseNode.SetMaxQueueLength(paramtable.Get().DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32()) + baseNode.SetMaxParallelism(paramtable.Get().DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32()) dd := &ddNode{ ctx: ctx, BaseNode: baseNode, collectionID: collID, - sealedSegInfo: make(map[UniqueID]*datapb.SegmentInfo, len(sealedSegments)), - growingSegInfo: make(map[UniqueID]*datapb.SegmentInfo, len(growingSegments)), + sealedSegInfo: make(map[util.UniqueID]*datapb.SegmentInfo, len(sealedSegments)), + growingSegInfo: make(map[util.UniqueID]*datapb.SegmentInfo, len(growingSegments)), droppedSegmentIDs: droppedSegmentIDs, vChannelName: vChannelName, compactionExecutor: executor, diff --git a/internal/datanode/flow_graph_dd_node_test.go b/internal/datanode/pipeline/flow_graph_dd_node_test.go similarity index 85% rename from internal/datanode/flow_graph_dd_node_test.go rename to internal/datanode/pipeline/flow_graph_dd_node_test.go index fc80d5f49bfd7..6f392f9c556a5 100644 --- a/internal/datanode/flow_graph_dd_node_test.go +++ b/internal/datanode/pipeline/flow_graph_dd_node_test.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package pipeline import ( "context" @@ -27,6 +27,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/datanode/compaction" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/pkg/mq/msgstream" @@ -63,9 +64,9 @@ func TestFlowGraph_DDNode_newDDNode(t *testing.T) { } var ( - collectionID = UniqueID(1) + collectionID = util.UniqueID(1) channelName = fmt.Sprintf("by-dev-rootcoord-dml-%s", t.Name()) - droppedSegIDs = []UniqueID{} + droppedSegIDs = []util.UniqueID{} ) for _, test := range tests { @@ -102,11 +103,11 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) { "Invalid input length == 0", }, { - []Msg{&flowGraphMsg{}, &flowGraphMsg{}, &flowGraphMsg{}}, + []Msg{&FlowGraphMsg{}, &FlowGraphMsg{}, &FlowGraphMsg{}}, "Invalid input length == 3", }, { - []Msg{&flowGraphMsg{}}, + []Msg{&FlowGraphMsg{}}, "Invalid input length == 1 but input message is not msgStreamMsg", }, } @@ -119,9 +120,9 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) { } // valid inputs tests := []struct { - ddnCollID UniqueID + ddnCollID util.UniqueID - msgCollID UniqueID + msgCollID util.UniqueID expectedChlen int description string @@ -158,7 +159,7 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) { if test.ddnCollID == test.msgCollID { assert.NotEmpty(t, rt) - assert.True(t, rt[0].(*flowGraphMsg).dropCollection) + assert.True(t, rt[0].(*FlowGraphMsg).dropCollection) } else { assert.NotEmpty(t, rt) } @@ -169,22 +170,22 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) { t.Run("Test DDNode Operate DropPartition Msg", func(t *testing.T) { // valid inputs tests := []struct { - ddnCollID UniqueID + ddnCollID util.UniqueID - msgCollID UniqueID - msgPartID UniqueID - expectOutput []UniqueID + msgCollID util.UniqueID + msgPartID util.UniqueID + expectOutput []util.UniqueID description string }{ { 1, 1, 101, - []UniqueID{101}, + []util.UniqueID{101}, "DropCollectionMsg collID == ddNode collID", }, { 1, 2, 101, - []UniqueID{}, + []util.UniqueID{}, "DropCollectionMsg collID != ddNode collID", }, } @@ -211,7 +212,7 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) { rt := ddn.Operate([]Msg{msgStreamMsg}) assert.NotEmpty(t, rt) - fgMsg, ok := rt[0].(*flowGraphMsg) + fgMsg, ok := rt[0].(*FlowGraphMsg) assert.True(t, ok) assert.ElementsMatch(t, test.expectOutput, fgMsg.dropPartitions) }) @@ -219,27 +220,27 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) { }) t.Run("Test DDNode Operate and filter insert msg", func(t *testing.T) { - var collectionID UniqueID = 1 + var collectionID util.UniqueID = 1 // Prepare ddNode states ddn := ddNode{ ctx: context.Background(), collectionID: collectionID, - droppedSegmentIDs: []UniqueID{100}, + droppedSegmentIDs: []util.UniqueID{100}, } tsMessages := []msgstream.TsMsg{getInsertMsg(100, 10000), getInsertMsg(200, 20000)} var msgStreamMsg Msg = flowgraph.GenerateMsgStreamMsg(tsMessages, 0, 0, nil, nil) rt := ddn.Operate([]Msg{msgStreamMsg}) - assert.Equal(t, 1, len(rt[0].(*flowGraphMsg).insertMessages)) + assert.Equal(t, 1, len(rt[0].(*FlowGraphMsg).InsertMessages)) }) t.Run("Test DDNode Operate Delete Msg", func(t *testing.T) { tests := []struct { - ddnCollID UniqueID - inMsgCollID UniqueID + ddnCollID util.UniqueID + inMsgCollID util.UniqueID - MsgEndTs Timestamp + MsgEndTs util.Timestamp expectedRtLen int description string @@ -272,7 +273,7 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) { // Test rt := ddn.Operate([]Msg{msgStreamMsg}) - assert.Equal(t, test.expectedRtLen, len(rt[0].(*flowGraphMsg).deleteMessages)) + assert.Equal(t, test.expectedRtLen, len(rt[0].(*FlowGraphMsg).DeleteMessages)) }) } }) @@ -282,16 +283,16 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { tests := []struct { description string - droppedSegIDs []UniqueID - sealedSegInfo map[UniqueID]*datapb.SegmentInfo - growingSegInfo map[UniqueID]*datapb.SegmentInfo + droppedSegIDs []util.UniqueID + sealedSegInfo map[util.UniqueID]*datapb.SegmentInfo + growingSegInfo map[util.UniqueID]*datapb.SegmentInfo inMsg *msgstream.InsertMsg expected bool }{ { "test dropped segments true", - []UniqueID{100}, + []util.UniqueID{100}, nil, nil, getInsertMsg(100, 10000), @@ -299,7 +300,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { }, { "test dropped segments true 2", - []UniqueID{100, 101, 102}, + []util.UniqueID{100, 101, 102}, nil, nil, getInsertMsg(102, 10000), @@ -307,8 +308,8 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { }, { "test sealed segments msgTs <= segmentTs true", - []UniqueID{}, - map[UniqueID]*datapb.SegmentInfo{ + []util.UniqueID{}, + map[util.UniqueID]*datapb.SegmentInfo{ 200: getSegmentInfo(200, 50000), 300: getSegmentInfo(300, 50000), }, @@ -318,8 +319,8 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { }, { "test sealed segments msgTs <= segmentTs true", - []UniqueID{}, - map[UniqueID]*datapb.SegmentInfo{ + []util.UniqueID{}, + map[util.UniqueID]*datapb.SegmentInfo{ 200: getSegmentInfo(200, 50000), 300: getSegmentInfo(300, 50000), }, @@ -329,8 +330,8 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { }, { "test sealed segments msgTs > segmentTs false", - []UniqueID{}, - map[UniqueID]*datapb.SegmentInfo{ + []util.UniqueID{}, + map[util.UniqueID]*datapb.SegmentInfo{ 200: getSegmentInfo(200, 50000), 300: getSegmentInfo(300, 50000), }, @@ -340,9 +341,9 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { }, { "test growing segments msgTs <= segmentTs true", - []UniqueID{}, + []util.UniqueID{}, nil, - map[UniqueID]*datapb.SegmentInfo{ + map[util.UniqueID]*datapb.SegmentInfo{ 200: getSegmentInfo(200, 50000), 300: getSegmentInfo(300, 50000), }, @@ -351,9 +352,9 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { }, { "test growing segments msgTs > segmentTs false", - []UniqueID{}, + []util.UniqueID{}, nil, - map[UniqueID]*datapb.SegmentInfo{ + map[util.UniqueID]*datapb.SegmentInfo{ 200: getSegmentInfo(200, 50000), 300: getSegmentInfo(300, 50000), }, @@ -362,12 +363,12 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { }, { "test not exist", - []UniqueID{}, - map[UniqueID]*datapb.SegmentInfo{ + []util.UniqueID{}, + map[util.UniqueID]*datapb.SegmentInfo{ 400: getSegmentInfo(500, 50000), 500: getSegmentInfo(400, 50000), }, - map[UniqueID]*datapb.SegmentInfo{ + map[util.UniqueID]*datapb.SegmentInfo{ 200: getSegmentInfo(200, 50000), 300: getSegmentInfo(300, 50000), }, @@ -377,7 +378,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { // for pChannel reuse on same collection { "test insert msg with different channelName", - []UniqueID{100}, + []util.UniqueID{100}, nil, nil, getInsertMsgWithChannel(100, 10000, anotherChannelName), @@ -405,10 +406,10 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { description string segRemained bool - segTs Timestamp - msgTs Timestamp + segTs util.Timestamp + msgTs util.Timestamp - sealedSegInfo map[UniqueID]*datapb.SegmentInfo + sealedSegInfo map[util.UniqueID]*datapb.SegmentInfo inMsg *msgstream.InsertMsg msgFiltered bool }{ @@ -417,7 +418,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { true, 50000, 10000, - map[UniqueID]*datapb.SegmentInfo{ + map[util.UniqueID]*datapb.SegmentInfo{ 100: getSegmentInfo(100, 50000), 101: getSegmentInfo(101, 50000), }, @@ -429,7 +430,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { true, 50000, 10000, - map[UniqueID]*datapb.SegmentInfo{ + map[util.UniqueID]*datapb.SegmentInfo{ 100: getSegmentInfo(100, 50000), 101: getSegmentInfo(101, 50000), }, @@ -441,7 +442,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { false, 50000, 10000, - map[UniqueID]*datapb.SegmentInfo{ + map[util.UniqueID]*datapb.SegmentInfo{ 100: getSegmentInfo(100, 70000), 101: getSegmentInfo(101, 50000), }, @@ -474,14 +475,14 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { description string segRemained bool - growingSegInfo map[UniqueID]*datapb.SegmentInfo + growingSegInfo map[util.UniqueID]*datapb.SegmentInfo inMsg *msgstream.InsertMsg msgFiltered bool }{ { "msgTssegTs", false, - map[UniqueID]*datapb.SegmentInfo{ + map[util.UniqueID]*datapb.SegmentInfo{ 100: getSegmentInfo(100, 50000), 101: getSegmentInfo(101, 50000), }, @@ -535,7 +536,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { func TestFlowGraph_DDNode_isDropped(t *testing.T) { tests := []struct { indroppedSegment []*datapb.SegmentInfo - inSeg UniqueID + inSeg util.UniqueID expectedOut bool @@ -581,18 +582,18 @@ func TestFlowGraph_DDNode_isDropped(t *testing.T) { } } -func getSegmentInfo(segmentID UniqueID, ts Timestamp) *datapb.SegmentInfo { +func getSegmentInfo(segmentID util.UniqueID, ts util.Timestamp) *datapb.SegmentInfo { return &datapb.SegmentInfo{ ID: segmentID, DmlPosition: &msgpb.MsgPosition{Timestamp: ts}, } } -func getInsertMsg(segmentID UniqueID, ts Timestamp) *msgstream.InsertMsg { +func getInsertMsg(segmentID util.UniqueID, ts util.Timestamp) *msgstream.InsertMsg { return getInsertMsgWithChannel(segmentID, ts, ddNodeChannelName) } -func getInsertMsgWithChannel(segmentID UniqueID, ts Timestamp, vChannelName string) *msgstream.InsertMsg { +func getInsertMsgWithChannel(segmentID util.UniqueID, ts util.Timestamp, vChannelName string) *msgstream.InsertMsg { return &msgstream.InsertMsg{ BaseMsg: msgstream.BaseMsg{EndTimestamp: ts}, InsertRequest: msgpb.InsertRequest{ diff --git a/internal/datanode/flow_graph_dmstream_input_node.go b/internal/datanode/pipeline/flow_graph_dmstream_input_node.go similarity index 95% rename from internal/datanode/flow_graph_dmstream_input_node.go rename to internal/datanode/pipeline/flow_graph_dmstream_input_node.go index 21242e2a5cfb8..be7d12347a1f0 100644 --- a/internal/datanode/flow_graph_dmstream_input_node.go +++ b/internal/datanode/pipeline/flow_graph_dmstream_input_node.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package pipeline import ( "context" @@ -66,8 +66,8 @@ func newDmInputNode(initCtx context.Context, dispatcherClient msgdispatcher.Clie node := flowgraph.NewInputNode( input, name, - Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32(), - Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32(), + paramtable.Get().DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32(), + paramtable.Get().DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32(), typeutil.DataNodeRole, dmNodeConfig.serverID, dmNodeConfig.collectionID, diff --git a/internal/datanode/flow_graph_dmstream_input_node_test.go b/internal/datanode/pipeline/flow_graph_dmstream_input_node_test.go similarity index 99% rename from internal/datanode/flow_graph_dmstream_input_node_test.go rename to internal/datanode/pipeline/flow_graph_dmstream_input_node_test.go index ae804fe19e5d2..3c81a358911c3 100644 --- a/internal/datanode/flow_graph_dmstream_input_node_test.go +++ b/internal/datanode/pipeline/flow_graph_dmstream_input_node_test.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package pipeline import ( "context" diff --git a/internal/datanode/flow_graph_manager.go b/internal/datanode/pipeline/flow_graph_manager.go similarity index 63% rename from internal/datanode/flow_graph_manager.go rename to internal/datanode/pipeline/flow_graph_manager.go index 390a92251b97d..659ddc3654e2a 100644 --- a/internal/datanode/flow_graph_manager.go +++ b/internal/datanode/pipeline/flow_graph_manager.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package pipeline import ( "context" @@ -22,8 +22,7 @@ import ( "go.uber.org/zap" - "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/util/paramtable" @@ -31,14 +30,13 @@ import ( ) type FlowgraphManager interface { - AddFlowgraph(ds *dataSyncService) - AddandStartWithEtcdTickler(dn *DataNode, vchan *datapb.VchannelInfo, schema *schemapb.CollectionSchema, tickler *etcdTickler) error + AddFlowgraph(ds *DataSyncService) RemoveFlowgraph(channel string) ClearFlowgraphs() - GetFlowgraphService(channel string) (*dataSyncService, bool) + GetFlowgraphService(channel string) (*DataSyncService, bool) HasFlowgraph(channel string) bool - HasFlowgraphWithOpID(channel string, opID UniqueID) bool + HasFlowgraphWithOpID(channel string, opID int64) bool GetFlowgraphCount() int GetCollectionIDs() []int64 @@ -50,58 +48,36 @@ var _ FlowgraphManager = (*fgManagerImpl)(nil) type fgManagerImpl struct { ctx context.Context cancelFunc context.CancelFunc - flowgraphs *typeutil.ConcurrentMap[string, *dataSyncService] + flowgraphs *typeutil.ConcurrentMap[string, *DataSyncService] } -func newFlowgraphManager() *fgManagerImpl { +func NewFlowgraphManager() *fgManagerImpl { ctx, cancelFunc := context.WithCancel(context.TODO()) return &fgManagerImpl{ ctx: ctx, cancelFunc: cancelFunc, - flowgraphs: typeutil.NewConcurrentMap[string, *dataSyncService](), + flowgraphs: typeutil.NewConcurrentMap[string, *DataSyncService](), } } -func (fm *fgManagerImpl) AddFlowgraph(ds *dataSyncService) { +func (fm *fgManagerImpl) AddFlowgraph(ds *DataSyncService) { fm.flowgraphs.Insert(ds.vchannelName, ds) metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Inc() } -func (fm *fgManagerImpl) AddandStartWithEtcdTickler(dn *DataNode, vchan *datapb.VchannelInfo, schema *schemapb.CollectionSchema, tickler *etcdTickler) error { - log := log.With(zap.String("channel", vchan.GetChannelName())) - if fm.flowgraphs.Contain(vchan.GetChannelName()) { - log.Warn("try to add an existed DataSyncService") - return nil - } - - dataSyncService, err := newServiceWithEtcdTickler(fm.ctx, dn, &datapb.ChannelWatchInfo{ - Schema: schema, - Vchan: vchan, - }, tickler) - if err != nil { - log.Warn("fail to create new DataSyncService", zap.Error(err)) - return err - } - dataSyncService.start() - fm.flowgraphs.Insert(vchan.GetChannelName(), dataSyncService) - - metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Inc() - return nil -} - func (fm *fgManagerImpl) RemoveFlowgraph(channel string) { if fg, loaded := fm.flowgraphs.Get(channel); loaded { fg.close() fm.flowgraphs.Remove(channel) metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Dec() - rateCol.removeFlowGraphChannel(channel) + util.RateCol.RemoveFlowGraphChannel(channel) } } func (fm *fgManagerImpl) ClearFlowgraphs() { log.Info("start drop all flowgraph resources in DataNode") - fm.flowgraphs.Range(func(key string, value *dataSyncService) bool { + fm.flowgraphs.Range(func(key string, value *DataSyncService) bool { value.GracefullyClose() fm.flowgraphs.GetAndRemove(key) @@ -110,7 +86,7 @@ func (fm *fgManagerImpl) ClearFlowgraphs() { }) } -func (fm *fgManagerImpl) GetFlowgraphService(channel string) (*dataSyncService, bool) { +func (fm *fgManagerImpl) GetFlowgraphService(channel string) (*DataSyncService, bool) { return fm.flowgraphs.Get(channel) } @@ -119,7 +95,7 @@ func (fm *fgManagerImpl) HasFlowgraph(channel string) bool { return exist } -func (fm *fgManagerImpl) HasFlowgraphWithOpID(channel string, opID UniqueID) bool { +func (fm *fgManagerImpl) HasFlowgraphWithOpID(channel string, opID util.UniqueID) bool { ds, exist := fm.flowgraphs.Get(channel) return exist && ds.opID == opID } @@ -131,7 +107,7 @@ func (fm *fgManagerImpl) GetFlowgraphCount() int { func (fm *fgManagerImpl) GetCollectionIDs() []int64 { collectionSet := typeutil.UniqueSet{} - fm.flowgraphs.Range(func(key string, value *dataSyncService) bool { + fm.flowgraphs.Range(func(key string, value *DataSyncService) bool { collectionSet.Insert(value.metacache.Collection()) return true }) diff --git a/internal/datanode/pipeline/flow_graph_manager_test.go b/internal/datanode/pipeline/flow_graph_manager_test.go new file mode 100644 index 0000000000000..5d4b67c12678b --- /dev/null +++ b/internal/datanode/pipeline/flow_graph_manager_test.go @@ -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, + } +} diff --git a/internal/datanode/flow_graph_message.go b/internal/datanode/pipeline/flow_graph_message.go similarity index 67% rename from internal/datanode/flow_graph_message.go rename to internal/datanode/pipeline/flow_graph_message.go index c14603529904b..ca2b72765e4ca 100644 --- a/internal/datanode/flow_graph_message.go +++ b/internal/datanode/pipeline/flow_graph_message.go @@ -14,10 +14,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package pipeline import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/pkg/mq/msgstream" @@ -42,38 +43,24 @@ type ( Blob = storage.Blob ) -type flowGraphMsg struct { +type FlowGraphMsg struct { BaseMsg - insertMessages []*msgstream.InsertMsg - deleteMessages []*msgstream.DeleteMsg - timeRange TimeRange - startPositions []*msgpb.MsgPosition - endPositions []*msgpb.MsgPosition + InsertMessages []*msgstream.InsertMsg + DeleteMessages []*msgstream.DeleteMsg + TimeRange util.TimeRange + StartPositions []*msgpb.MsgPosition + EndPositions []*msgpb.MsgPosition + // segmentsToSync is the signal used by insertBufferNode to notify deleteNode to flush - segmentsToSync []UniqueID + segmentsToSync []util.UniqueID dropCollection bool - dropPartitions []UniqueID + dropPartitions []util.UniqueID } -func (fgMsg *flowGraphMsg) TimeTick() Timestamp { - return fgMsg.timeRange.timestampMax +func (fgMsg *FlowGraphMsg) TimeTick() util.Timestamp { + return fgMsg.TimeRange.TimestampMax } -func (fgMsg *flowGraphMsg) IsClose() bool { +func (fgMsg *FlowGraphMsg) IsClose() bool { return fgMsg.BaseMsg.IsCloseMsg() } - -// flush Msg is used in flowgraph insertBufferNode to flush the given segment -type flushMsg struct { - msgID UniqueID - timestamp Timestamp - segmentID UniqueID - collectionID UniqueID - // isFlush illustrates if this is a flush or normal sync - isFlush bool -} - -type resendTTMsg struct { - msgID UniqueID - segmentIDs []UniqueID -} diff --git a/internal/datanode/flow_graph_message_test.go b/internal/datanode/pipeline/flow_graph_message_test.go similarity index 85% rename from internal/datanode/flow_graph_message_test.go rename to internal/datanode/pipeline/flow_graph_message_test.go index d5d9dbbd6cdb3..74e2f387adee5 100644 --- a/internal/datanode/flow_graph_message_test.go +++ b/internal/datanode/pipeline/flow_graph_message_test.go @@ -14,17 +14,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package pipeline import ( "testing" "github.com/stretchr/testify/assert" + + "github.com/milvus-io/milvus/internal/datanode/util" ) func TestInsertMsg_TimeTick(te *testing.T) { tests := []struct { - timeTimestanpMax Timestamp + timeTimestanpMax util.Timestamp description string }{ @@ -34,7 +36,7 @@ func TestInsertMsg_TimeTick(te *testing.T) { for _, test := range tests { te.Run(test.description, func(t *testing.T) { - fgMsg := &flowGraphMsg{timeRange: TimeRange{timestampMax: test.timeTimestanpMax}} + fgMsg := &FlowGraphMsg{TimeRange: util.TimeRange{TimestampMax: test.timeTimestanpMax}} assert.Equal(t, test.timeTimestanpMax, fgMsg.TimeTick()) }) } diff --git a/internal/datanode/flow_graph_node.go b/internal/datanode/pipeline/flow_graph_node.go similarity index 98% rename from internal/datanode/flow_graph_node.go rename to internal/datanode/pipeline/flow_graph_node.go index f91036f76cb73..3c8a246b2cfb9 100644 --- a/internal/datanode/flow_graph_node.go +++ b/internal/datanode/pipeline/flow_graph_node.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package pipeline import ( "sync/atomic" diff --git a/internal/datanode/flow_graph_time_tick_node.go b/internal/datanode/pipeline/flow_graph_time_tick_node.go similarity index 89% rename from internal/datanode/flow_graph_time_tick_node.go rename to internal/datanode/pipeline/flow_graph_time_tick_node.go index 28a3ee0149928..1e6fddd9bf6d5 100644 --- a/internal/datanode/flow_graph_time_tick_node.go +++ b/internal/datanode/pipeline/flow_graph_time_tick_node.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package pipeline import ( "fmt" @@ -26,6 +26,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/datanode/metacache" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/datanode/writebuffer" "github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/pkg/log" @@ -42,7 +43,7 @@ type ttNode struct { metacache metacache.MetaCache writeBufferManager writebuffer.BufferManager lastUpdateTime *atomic.Time - cpUpdater *channelCheckpointUpdater + cpUpdater *util.ChannelCheckpointUpdater dropMode *atomic.Bool } @@ -55,7 +56,7 @@ func (ttn *ttNode) IsValidInMsg(in []Msg) bool { if !ttn.BaseNode.IsValidInMsg(in) { return false } - _, ok := in[0].(*flowGraphMsg) + _, ok := in[0].(*FlowGraphMsg) if !ok { log.Warn("type assertion failed for flowGraphMsg", zap.String("name", reflect.TypeOf(in[0]).Name())) return false @@ -68,7 +69,7 @@ func (ttn *ttNode) Close() { // Operate handles input messages, implementing flowgraph.Node func (ttn *ttNode) Operate(in []Msg) []Msg { - fgMsg := in[0].(*flowGraphMsg) + fgMsg := in[0].(*FlowGraphMsg) if fgMsg.dropCollection { ttn.dropMode.Store(true) } @@ -80,9 +81,9 @@ func (ttn *ttNode) Operate(in []Msg) []Msg { return []Msg{} } - curTs, _ := tsoutil.ParseTS(fgMsg.timeRange.timestampMax) + curTs, _ := tsoutil.ParseTS(fgMsg.TimeRange.TimestampMax) if fgMsg.IsCloseMsg() { - if len(fgMsg.endPositions) > 0 { + if len(fgMsg.EndPositions) > 0 { channelPos, _, err := ttn.writeBufferManager.GetCheckpoint(ttn.vChannelName) if err != nil { log.Warn("channel removed", zap.String("channel", ttn.vChannelName), zap.Error(err)) @@ -128,10 +129,10 @@ func (ttn *ttNode) updateChannelCP(channelPos *msgpb.MsgPosition, curTs time.Tim ttn.lastUpdateTime.Store(curTs) } -func newTTNode(config *nodeConfig, wbManager writebuffer.BufferManager, cpUpdater *channelCheckpointUpdater) (*ttNode, error) { +func newTTNode(config *nodeConfig, wbManager writebuffer.BufferManager, cpUpdater *util.ChannelCheckpointUpdater) (*ttNode, error) { baseNode := BaseNode{} - baseNode.SetMaxQueueLength(Params.DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32()) - baseNode.SetMaxParallelism(Params.DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32()) + baseNode.SetMaxQueueLength(paramtable.Get().DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32()) + baseNode.SetMaxParallelism(paramtable.Get().DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32()) tt := &ttNode{ BaseNode: baseNode, diff --git a/internal/datanode/flow_graph_time_ticker.go b/internal/datanode/pipeline/flow_graph_time_ticker.go similarity index 81% rename from internal/datanode/flow_graph_time_ticker.go rename to internal/datanode/pipeline/flow_graph_time_ticker.go index 039a7c07cb7ae..c793001305170 100644 --- a/internal/datanode/flow_graph_time_ticker.go +++ b/internal/datanode/pipeline/flow_graph_time_ticker.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package pipeline import ( "sync" @@ -24,10 +24,12 @@ import ( "go.uber.org/zap" "golang.org/x/exp/maps" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/paramtable" ) -type sendTimeTick func(Timestamp, []int64) error +type sendTimeTick func(util.Timestamp, []int64) error // mergedTimeTickerSender reduces time ticker sending rate when datanode is doing `fast-forwarding` // it makes sure time ticker send at most 10 times a second (1tick/100millisecond) @@ -46,11 +48,6 @@ type mergedTimeTickerSender struct { closeOnce sync.Once } -var ( - uniqueMergedTimeTickerSender *mergedTimeTickerSender - getUniqueMergedTimeTickerSender sync.Once -) - func newUniqueMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender { return &mergedTimeTickerSender{ ts: 0, // 0 for not tt send @@ -61,17 +58,7 @@ func newUniqueMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender } } -func getOrCreateMergedTimeTickerSender(send sendTimeTick) *mergedTimeTickerSender { - getUniqueMergedTimeTickerSender.Do(func() { - uniqueMergedTimeTickerSender = newUniqueMergedTimeTickerSender(send) - uniqueMergedTimeTickerSender.wg.Add(2) - go uniqueMergedTimeTickerSender.tick() - go uniqueMergedTimeTickerSender.work() - }) - return uniqueMergedTimeTickerSender -} - -func (mt *mergedTimeTickerSender) bufferTs(ts Timestamp, segmentIDs []int64) { +func (mt *mergedTimeTickerSender) bufferTs(ts util.Timestamp, segmentIDs []int64) { mt.mu.Lock() defer mt.mu.Unlock() mt.ts = ts @@ -83,7 +70,7 @@ func (mt *mergedTimeTickerSender) bufferTs(ts Timestamp, segmentIDs []int64) { func (mt *mergedTimeTickerSender) tick() { defer mt.wg.Done() // this duration might be configuable in the future - t := time.NewTicker(Params.DataNodeCfg.DataNodeTimeTickInterval.GetAsDuration(time.Millisecond)) // 500 millisecond + t := time.NewTicker(paramtable.Get().DataNodeCfg.DataNodeTimeTickInterval.GetAsDuration(time.Millisecond)) // 500 millisecond defer t.Stop() for { select { diff --git a/internal/datanode/flow_graph_write_node.go b/internal/datanode/pipeline/flow_graph_write_node.go similarity index 69% rename from internal/datanode/flow_graph_write_node.go rename to internal/datanode/pipeline/flow_graph_write_node.go index 657db18a9a905..27f53617a66c5 100644 --- a/internal/datanode/flow_graph_write_node.go +++ b/internal/datanode/pipeline/flow_graph_write_node.go @@ -1,4 +1,4 @@ -package datanode +package pipeline import ( "context" @@ -11,6 +11,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/datanode/metacache" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/datanode/writebuffer" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" @@ -27,7 +28,7 @@ type writeNode struct { } func (wNode *writeNode) Operate(in []Msg) []Msg { - fgMsg := in[0].(*flowGraphMsg) + fgMsg := in[0].(*FlowGraphMsg) // close msg, ignore all data if fgMsg.IsCloseMsg() { @@ -35,31 +36,31 @@ func (wNode *writeNode) Operate(in []Msg) []Msg { } // replace pchannel with vchannel - startPositions := make([]*msgpb.MsgPosition, 0, len(fgMsg.startPositions)) - for idx := range fgMsg.startPositions { - pos := proto.Clone(fgMsg.startPositions[idx]).(*msgpb.MsgPosition) + startPositions := make([]*msgpb.MsgPosition, 0, len(fgMsg.StartPositions)) + for idx := range fgMsg.StartPositions { + pos := proto.Clone(fgMsg.StartPositions[idx]).(*msgpb.MsgPosition) pos.ChannelName = wNode.channelName startPositions = append(startPositions, pos) } - fgMsg.startPositions = startPositions - endPositions := make([]*msgpb.MsgPosition, 0, len(fgMsg.endPositions)) - for idx := range fgMsg.endPositions { - pos := proto.Clone(fgMsg.endPositions[idx]).(*msgpb.MsgPosition) + fgMsg.StartPositions = startPositions + endPositions := make([]*msgpb.MsgPosition, 0, len(fgMsg.EndPositions)) + for idx := range fgMsg.EndPositions { + pos := proto.Clone(fgMsg.EndPositions[idx]).(*msgpb.MsgPosition) pos.ChannelName = wNode.channelName endPositions = append(endPositions, pos) } - fgMsg.endPositions = endPositions + fgMsg.EndPositions = endPositions - if len(fgMsg.startPositions) == 0 { + if len(fgMsg.StartPositions) == 0 { return []Msg{} } - if len(fgMsg.endPositions) == 0 { + if len(fgMsg.EndPositions) == 0 { return []Msg{} } var spans []trace.Span - for _, msg := range fgMsg.insertMessages { - ctx, sp := startTracer(msg, "WriteNode") + for _, msg := range fgMsg.InsertMessages { + ctx, sp := util.StartTracer(msg, "WriteNode") spans = append(spans, sp) msg.SetTraceCtx(ctx) } @@ -69,16 +70,16 @@ func (wNode *writeNode) Operate(in []Msg) []Msg { } }() - start, end := fgMsg.startPositions[0], fgMsg.endPositions[0] + start, end := fgMsg.StartPositions[0], fgMsg.EndPositions[0] - err := wNode.wbManager.BufferData(wNode.channelName, fgMsg.insertMessages, fgMsg.deleteMessages, start, end) + err := wNode.wbManager.BufferData(wNode.channelName, fgMsg.InsertMessages, fgMsg.DeleteMessages, start, end) if err != nil { log.Error("failed to buffer data", zap.Error(err)) panic(err) } stats := lo.FilterMap( - lo.Keys(lo.SliceToMap(fgMsg.insertMessages, func(msg *msgstream.InsertMsg) (int64, struct{}) { return msg.GetSegmentID(), struct{}{} })), + lo.Keys(lo.SliceToMap(fgMsg.InsertMessages, func(msg *msgstream.InsertMsg) (int64, struct{}) { return msg.GetSegmentID(), struct{}{} })), func(id int64, _ int) (*commonpb.SegmentStats, bool) { segInfo, ok := wNode.metacache.GetSegmentByID(id) if !ok { @@ -91,12 +92,12 @@ func (wNode *writeNode) Operate(in []Msg) []Msg { }, true }) - wNode.updater.update(wNode.channelName, end.GetTimestamp(), stats) + wNode.updater.Update(wNode.channelName, end.GetTimestamp(), stats) - res := flowGraphMsg{ - timeRange: fgMsg.timeRange, - startPositions: fgMsg.startPositions, - endPositions: fgMsg.endPositions, + res := FlowGraphMsg{ + TimeRange: fgMsg.TimeRange, + StartPositions: fgMsg.StartPositions, + EndPositions: fgMsg.EndPositions, dropCollection: fgMsg.dropCollection, } diff --git a/internal/datanode/mock_fgmanager.go b/internal/datanode/pipeline/mock_fgmanager.go similarity index 79% rename from internal/datanode/mock_fgmanager.go rename to internal/datanode/pipeline/mock_fgmanager.go index c31835d36365f..6945e21ff271f 100644 --- a/internal/datanode/mock_fgmanager.go +++ b/internal/datanode/pipeline/mock_fgmanager.go @@ -1,13 +1,8 @@ // Code generated by mockery v2.32.4. DO NOT EDIT. -package datanode +package pipeline -import ( - datapb "github.com/milvus-io/milvus/internal/proto/datapb" - mock "github.com/stretchr/testify/mock" - - schemapb "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" -) +import mock "github.com/stretchr/testify/mock" // MockFlowgraphManager is an autogenerated mock type for the FlowgraphManager type type MockFlowgraphManager struct { @@ -23,7 +18,7 @@ func (_m *MockFlowgraphManager) EXPECT() *MockFlowgraphManager_Expecter { } // AddFlowgraph provides a mock function with given fields: ds -func (_m *MockFlowgraphManager) AddFlowgraph(ds *dataSyncService) { +func (_m *MockFlowgraphManager) AddFlowgraph(ds *DataSyncService) { _m.Called(ds) } @@ -33,14 +28,14 @@ type MockFlowgraphManager_AddFlowgraph_Call struct { } // AddFlowgraph is a helper method to define mock.On call -// - ds *dataSyncService +// - ds *DataSyncService func (_e *MockFlowgraphManager_Expecter) AddFlowgraph(ds interface{}) *MockFlowgraphManager_AddFlowgraph_Call { return &MockFlowgraphManager_AddFlowgraph_Call{Call: _e.mock.On("AddFlowgraph", ds)} } -func (_c *MockFlowgraphManager_AddFlowgraph_Call) Run(run func(ds *dataSyncService)) *MockFlowgraphManager_AddFlowgraph_Call { +func (_c *MockFlowgraphManager_AddFlowgraph_Call) Run(run func(ds *DataSyncService)) *MockFlowgraphManager_AddFlowgraph_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(*dataSyncService)) + run(args[0].(*DataSyncService)) }) return _c } @@ -50,52 +45,7 @@ func (_c *MockFlowgraphManager_AddFlowgraph_Call) Return() *MockFlowgraphManager return _c } -func (_c *MockFlowgraphManager_AddFlowgraph_Call) RunAndReturn(run func(*dataSyncService)) *MockFlowgraphManager_AddFlowgraph_Call { - _c.Call.Return(run) - return _c -} - -// AddandStartWithEtcdTickler provides a mock function with given fields: dn, vchan, schema, tickler -func (_m *MockFlowgraphManager) AddandStartWithEtcdTickler(dn *DataNode, vchan *datapb.VchannelInfo, schema *schemapb.CollectionSchema, tickler *etcdTickler) error { - ret := _m.Called(dn, vchan, schema, tickler) - - var r0 error - if rf, ok := ret.Get(0).(func(*DataNode, *datapb.VchannelInfo, *schemapb.CollectionSchema, *etcdTickler) error); ok { - r0 = rf(dn, vchan, schema, tickler) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// MockFlowgraphManager_AddandStartWithEtcdTickler_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'AddandStartWithEtcdTickler' -type MockFlowgraphManager_AddandStartWithEtcdTickler_Call struct { - *mock.Call -} - -// AddandStartWithEtcdTickler is a helper method to define mock.On call -// - dn *DataNode -// - vchan *datapb.VchannelInfo -// - schema *schemapb.CollectionSchema -// - tickler *etcdTickler -func (_e *MockFlowgraphManager_Expecter) AddandStartWithEtcdTickler(dn interface{}, vchan interface{}, schema interface{}, tickler interface{}) *MockFlowgraphManager_AddandStartWithEtcdTickler_Call { - return &MockFlowgraphManager_AddandStartWithEtcdTickler_Call{Call: _e.mock.On("AddandStartWithEtcdTickler", dn, vchan, schema, tickler)} -} - -func (_c *MockFlowgraphManager_AddandStartWithEtcdTickler_Call) Run(run func(dn *DataNode, vchan *datapb.VchannelInfo, schema *schemapb.CollectionSchema, tickler *etcdTickler)) *MockFlowgraphManager_AddandStartWithEtcdTickler_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(*DataNode), args[1].(*datapb.VchannelInfo), args[2].(*schemapb.CollectionSchema), args[3].(*etcdTickler)) - }) - return _c -} - -func (_c *MockFlowgraphManager_AddandStartWithEtcdTickler_Call) Return(_a0 error) *MockFlowgraphManager_AddandStartWithEtcdTickler_Call { - _c.Call.Return(_a0) - return _c -} - -func (_c *MockFlowgraphManager_AddandStartWithEtcdTickler_Call) RunAndReturn(run func(*DataNode, *datapb.VchannelInfo, *schemapb.CollectionSchema, *etcdTickler) error) *MockFlowgraphManager_AddandStartWithEtcdTickler_Call { +func (_c *MockFlowgraphManager_AddFlowgraph_Call) RunAndReturn(run func(*DataSyncService)) *MockFlowgraphManager_AddFlowgraph_Call { _c.Call.Return(run) return _c } @@ -249,19 +199,19 @@ func (_c *MockFlowgraphManager_GetFlowgraphCount_Call) RunAndReturn(run func() i } // GetFlowgraphService provides a mock function with given fields: channel -func (_m *MockFlowgraphManager) GetFlowgraphService(channel string) (*dataSyncService, bool) { +func (_m *MockFlowgraphManager) GetFlowgraphService(channel string) (*DataSyncService, bool) { ret := _m.Called(channel) - var r0 *dataSyncService + var r0 *DataSyncService var r1 bool - if rf, ok := ret.Get(0).(func(string) (*dataSyncService, bool)); ok { + if rf, ok := ret.Get(0).(func(string) (*DataSyncService, bool)); ok { return rf(channel) } - if rf, ok := ret.Get(0).(func(string) *dataSyncService); ok { + if rf, ok := ret.Get(0).(func(string) *DataSyncService); ok { r0 = rf(channel) } else { if ret.Get(0) != nil { - r0 = ret.Get(0).(*dataSyncService) + r0 = ret.Get(0).(*DataSyncService) } } @@ -292,12 +242,12 @@ func (_c *MockFlowgraphManager_GetFlowgraphService_Call) Run(run func(channel st return _c } -func (_c *MockFlowgraphManager_GetFlowgraphService_Call) Return(_a0 *dataSyncService, _a1 bool) *MockFlowgraphManager_GetFlowgraphService_Call { +func (_c *MockFlowgraphManager_GetFlowgraphService_Call) Return(_a0 *DataSyncService, _a1 bool) *MockFlowgraphManager_GetFlowgraphService_Call { _c.Call.Return(_a0, _a1) return _c } -func (_c *MockFlowgraphManager_GetFlowgraphService_Call) RunAndReturn(run func(string) (*dataSyncService, bool)) *MockFlowgraphManager_GetFlowgraphService_Call { +func (_c *MockFlowgraphManager_GetFlowgraphService_Call) RunAndReturn(run func(string) (*DataSyncService, bool)) *MockFlowgraphManager_GetFlowgraphService_Call { _c.Call.Return(run) return _c } diff --git a/internal/datanode/stats_updater.go b/internal/datanode/pipeline/stats_updater.go similarity index 88% rename from internal/datanode/stats_updater.go rename to internal/datanode/pipeline/stats_updater.go index 2f25c88136bba..264cfbcfa73b6 100644 --- a/internal/datanode/stats_updater.go +++ b/internal/datanode/pipeline/stats_updater.go @@ -1,4 +1,4 @@ -package datanode +package pipeline import ( "fmt" @@ -8,6 +8,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/mq/msgstream" "github.com/milvus-io/milvus/pkg/util/commonpbutil" @@ -16,7 +17,7 @@ import ( ) type statsUpdater interface { - update(channel string, ts Timestamp, stats []*commonpb.SegmentStats) + Update(channel string, ts util.Timestamp, stats []*commonpb.SegmentStats) } // mqStatsUpdater is the wrapper of mergedTimeTickSender @@ -40,7 +41,7 @@ func newMqStatsUpdater(config *nodeConfig, producer msgstream.MsgStream) statsUp return updater } -func (u *mqStatsUpdater) send(ts Timestamp, segmentIDs []int64) error { +func (u *mqStatsUpdater) send(ts util.Timestamp, segmentIDs []int64) error { u.mut.Lock() defer u.mut.Unlock() stats := lo.Map(segmentIDs, func(id int64, _ int) *commonpb.SegmentStats { @@ -86,7 +87,7 @@ func (u *mqStatsUpdater) send(ts Timestamp, segmentIDs []int64) error { return nil } -func (u *mqStatsUpdater) update(channel string, ts Timestamp, stats []*commonpb.SegmentStats) { +func (u *mqStatsUpdater) Update(channel string, ts util.Timestamp, stats []*commonpb.SegmentStats) { u.mut.Lock() defer u.mut.Unlock() segmentIDs := lo.Map(stats, func(stats *commonpb.SegmentStats, _ int) int64 { return stats.SegmentID }) diff --git a/internal/datanode/stats_updater_test.go b/internal/datanode/pipeline/stats_updater_test.go similarity index 98% rename from internal/datanode/stats_updater_test.go rename to internal/datanode/pipeline/stats_updater_test.go index b41dedfa1a897..952ac0c519976 100644 --- a/internal/datanode/stats_updater_test.go +++ b/internal/datanode/pipeline/stats_updater_test.go @@ -1,4 +1,4 @@ -package datanode +package pipeline import ( "testing" diff --git a/internal/datanode/services.go b/internal/datanode/services.go index 6608144016d48..f2046ff27c75d 100644 --- a/internal/datanode/services.go +++ b/internal/datanode/services.go @@ -32,7 +32,6 @@ import ( "github.com/milvus-io/milvus/internal/datanode/importv2" "github.com/milvus-io/milvus/internal/datanode/io" "github.com/milvus-io/milvus/internal/datanode/metacache" - "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/internalpb" @@ -306,7 +305,7 @@ func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegments allSegments[segID] = struct{}{} } - missingSegments := ds.metacache.DetectMissingSegments(allSegments) + missingSegments := ds.GetMetaCache().DetectMissingSegments(allSegments) newSegments := make([]*datapb.SyncSegmentInfo, 0, len(missingSegments)) futures := make([]*conc.Future[any], 0, len(missingSegments)) @@ -323,7 +322,7 @@ func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegments log.Warn("failed to DecompressBinLog", zap.Error(err)) return val, err } - pks, err := util.LoadStats(ctx, node.chunkManager, ds.metacache.Schema(), newSeg.GetSegmentId(), []*datapb.FieldBinlog{newSeg.GetPkStatsLog()}) + pks, err := compaction.LoadStats(ctx, node.chunkManager, ds.GetMetaCache().Schema(), newSeg.GetSegmentId(), []*datapb.FieldBinlog{newSeg.GetPkStatsLog()}) if err != nil { log.Warn("failed to load segment stats log", zap.Error(err)) return val, err @@ -343,7 +342,7 @@ func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegments return future.Value().(*metacache.BloomFilterSet) }) - ds.metacache.UpdateSegmentView(req.GetPartitionId(), newSegments, newSegmentsBF, allSegments) + ds.GetMetaCache().UpdateSegmentView(req.GetPartitionId(), newSegments, newSegmentsBF, allSegments) return merr.Success(), nil } diff --git a/internal/datanode/services_test.go b/internal/datanode/services_test.go index 52a2edb8a0444..7342eed06858a 100644 --- a/internal/datanode/services_test.go +++ b/internal/datanode/services_test.go @@ -36,6 +36,8 @@ import ( "github.com/milvus-io/milvus/internal/datanode/broker" "github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/datanode/metacache" + "github.com/milvus-io/milvus/internal/datanode/pipeline" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" @@ -77,7 +79,7 @@ func (s *DataNodeServicesSuite) SetupSuite() { } func (s *DataNodeServicesSuite) SetupTest() { - s.node = newIDLEDataNodeMock(s.ctx, schemapb.DataType_Int64) + s.node = NewIDLEDataNodeMock(s.ctx, schemapb.DataType_Int64) s.node.SetEtcdClient(s.etcdCli) err := s.node.Init() @@ -292,18 +294,25 @@ func (s *DataNodeServicesSuite) TestFlushSegments() { FlushedSegmentIds: []int64{}, } - err := s.node.flowgraphManager.AddandStartWithEtcdTickler(s.node, vchan, schema, genTestTickler()) - s.Require().NoError(err) - - fgservice, ok := s.node.flowgraphManager.GetFlowgraphService(dmChannelName) - s.Require().True(ok) + chanWathInfo := &datapb.ChannelWatchInfo{ + Vchan: vchan, + State: datapb.ChannelWatchState_WatchSuccess, + Schema: schema, + } metaCache := metacache.NewMockMetaCache(s.T()) metaCache.EXPECT().Collection().Return(1).Maybe() metaCache.EXPECT().Schema().Return(schema).Maybe() - s.node.writeBufferManager.Register(dmChannelName, metaCache, nil) - fgservice.metacache.AddSegment(&datapb.SegmentInfo{ + ds, err := pipeline.NewDataSyncService(context.TODO(), getPipelineParams(s.node), chanWathInfo, util.NewTickler()) + ds.GetMetaCache() + s.Require().NoError(err) + s.node.flowgraphManager.AddFlowgraph(ds) + + fgservice, ok := s.node.flowgraphManager.GetFlowgraphService(dmChannelName) + s.Require().True(ok) + + fgservice.GetMetaCache().AddSegment(&datapb.SegmentInfo{ ID: segmentID, CollectionID: 1, PartitionID: 2, @@ -419,7 +428,7 @@ func (s *DataNodeServicesSuite) TestShowConfigurations() { func (s *DataNodeServicesSuite) TestGetMetrics() { node := &DataNode{} node.SetSession(&sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 1}}) - node.flowgraphManager = newFlowgraphManager() + node.flowgraphManager = pipeline.NewFlowgraphManager() // server is closed node.stateCode.Store(commonpb.StateCode_Abnormal) resp, err := node.GetMetrics(s.ctx, &milvuspb.GetMetricsRequest{}) @@ -618,10 +627,9 @@ func (s *DataNodeServicesSuite) TestSyncSegments() { }, func(*datapb.SegmentInfo) *metacache.BloomFilterSet { return metacache.NewBloomFilterSet() }) - mockFlowgraphManager := NewMockFlowgraphManager(s.T()) - mockFlowgraphManager.EXPECT().GetFlowgraphService(mock.Anything).Return(&dataSyncService{ - metacache: cache, - }, true) + mockFlowgraphManager := pipeline.NewMockFlowgraphManager(s.T()) + mockFlowgraphManager.EXPECT().GetFlowgraphService(mock.Anything). + Return(pipeline.NewDataSyncServiceWithMetaCache(cache), true) s.node.flowgraphManager = mockFlowgraphManager ctx := context.Background() req := &datapb.SyncSegmentsRequest{ diff --git a/internal/datanode/cache.go b/internal/datanode/util/cache.go similarity index 93% rename from internal/datanode/cache.go rename to internal/datanode/util/cache.go index fde4b7e0be7dc..9da70319708c3 100644 --- a/internal/datanode/cache.go +++ b/internal/datanode/util/cache.go @@ -14,9 +14,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package util -import "github.com/milvus-io/milvus/pkg/util/typeutil" +import ( + "github.com/milvus-io/milvus/pkg/util/typeutil" +) // Cache stores flushing segments' ids to prevent flushing the same segment again and again. // @@ -29,8 +31,8 @@ type Cache struct { *typeutil.ConcurrentSet[UniqueID] } -// newCache returns a new Cache -func newCache() *Cache { +// NewCache returns a new Cache +func NewCache() *Cache { return &Cache{ ConcurrentSet: typeutil.NewConcurrentSet[UniqueID](), } diff --git a/internal/datanode/cache_test.go b/internal/datanode/util/cache_test.go similarity index 96% rename from internal/datanode/cache_test.go rename to internal/datanode/util/cache_test.go index 01776fee48419..d94c003021a06 100644 --- a/internal/datanode/cache_test.go +++ b/internal/datanode/util/cache_test.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package util import ( "testing" @@ -23,7 +23,7 @@ import ( ) func TestSegmentCache(t *testing.T) { - segCache := newCache() + segCache := NewCache() assert.False(t, segCache.checkIfCached(0)) diff --git a/internal/datanode/channel_checkpoint_updater.go b/internal/datanode/util/checkpoint_updater.go similarity index 90% rename from internal/datanode/channel_checkpoint_updater.go rename to internal/datanode/util/checkpoint_updater.go index bd6b31692ed29..99d70c5be6143 100644 --- a/internal/datanode/channel_checkpoint_updater.go +++ b/internal/datanode/util/checkpoint_updater.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package util import ( "context" @@ -41,7 +41,7 @@ type channelCPUpdateTask struct { flush bool // indicates whether the task originates from flush } -type channelCheckpointUpdater struct { +type ChannelCheckpointUpdater struct { broker broker.Broker mu sync.RWMutex @@ -52,8 +52,8 @@ type channelCheckpointUpdater struct { closeOnce sync.Once } -func newChannelCheckpointUpdater(broker broker.Broker) *channelCheckpointUpdater { - return &channelCheckpointUpdater{ +func NewChannelCheckpointUpdater(broker broker.Broker) *ChannelCheckpointUpdater { + return &ChannelCheckpointUpdater{ broker: broker, tasks: make(map[string]*channelCPUpdateTask), closeCh: make(chan struct{}), @@ -61,7 +61,7 @@ func newChannelCheckpointUpdater(broker broker.Broker) *channelCheckpointUpdater } } -func (ccu *channelCheckpointUpdater) start() { +func (ccu *ChannelCheckpointUpdater) Start() { log.Info("channel checkpoint updater start") ticker := time.NewTicker(paramtable.Get().DataNodeCfg.ChannelCheckpointUpdateTickInSeconds.GetAsDuration(time.Second)) defer ticker.Stop() @@ -90,21 +90,21 @@ func (ccu *channelCheckpointUpdater) start() { } } -func (ccu *channelCheckpointUpdater) getTask(channel string) (*channelCPUpdateTask, bool) { +func (ccu *ChannelCheckpointUpdater) getTask(channel string) (*channelCPUpdateTask, bool) { ccu.mu.RLock() defer ccu.mu.RUnlock() task, ok := ccu.tasks[channel] return task, ok } -func (ccu *channelCheckpointUpdater) trigger() { +func (ccu *ChannelCheckpointUpdater) trigger() { select { case ccu.notifyChan <- struct{}{}: default: } } -func (ccu *channelCheckpointUpdater) updateCheckpoints(tasks []*channelCPUpdateTask) { +func (ccu *ChannelCheckpointUpdater) updateCheckpoints(tasks []*channelCPUpdateTask) { taskGroups := lo.Chunk(tasks, paramtable.Get().DataNodeCfg.MaxChannelCheckpointsPerRPC.GetAsInt()) updateChanCPMaxParallel := paramtable.Get().DataNodeCfg.UpdateChannelCheckpointMaxParallel.GetAsInt() if updateChanCPMaxParallel <= 0 { @@ -152,7 +152,7 @@ func (ccu *channelCheckpointUpdater) updateCheckpoints(tasks []*channelCPUpdateT }) } -func (ccu *channelCheckpointUpdater) execute() { +func (ccu *ChannelCheckpointUpdater) execute() { ccu.mu.RLock() tasks := lo.Values(ccu.tasks) ccu.mu.RUnlock() @@ -160,7 +160,7 @@ func (ccu *channelCheckpointUpdater) execute() { ccu.updateCheckpoints(tasks) } -func (ccu *channelCheckpointUpdater) AddTask(channelPos *msgpb.MsgPosition, flush bool, callback func()) { +func (ccu *ChannelCheckpointUpdater) AddTask(channelPos *msgpb.MsgPosition, flush bool, callback func()) { if channelPos == nil || channelPos.GetMsgID() == nil || channelPos.GetChannelName() == "" { log.Warn("illegal checkpoint", zap.Any("pos", channelPos)) return @@ -201,13 +201,13 @@ func (ccu *channelCheckpointUpdater) AddTask(channelPos *msgpb.MsgPosition, flus } } -func (ccu *channelCheckpointUpdater) taskNum() int { +func (ccu *ChannelCheckpointUpdater) taskNum() int { ccu.mu.RLock() defer ccu.mu.RUnlock() return len(ccu.tasks) } -func (ccu *channelCheckpointUpdater) close() { +func (ccu *ChannelCheckpointUpdater) Close() { ccu.closeOnce.Do(func() { close(ccu.closeCh) }) diff --git a/internal/datanode/channel_checkpoint_updater_test.go b/internal/datanode/util/checkpoint_updater_test.go similarity index 94% rename from internal/datanode/channel_checkpoint_updater_test.go rename to internal/datanode/util/checkpoint_updater_test.go index 9fc52b3cb92d0..7e75d588b923e 100644 --- a/internal/datanode/channel_checkpoint_updater_test.go +++ b/internal/datanode/util/checkpoint_updater_test.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package util import ( "context" @@ -36,12 +36,12 @@ type ChannelCPUpdaterSuite struct { suite.Suite broker *broker.MockBroker - updater *channelCheckpointUpdater + updater *ChannelCheckpointUpdater } func (s *ChannelCPUpdaterSuite) SetupTest() { s.broker = broker.NewMockBroker(s.T()) - s.updater = newChannelCheckpointUpdater(s.broker) + s.updater = NewChannelCheckpointUpdater(s.broker) } func (s *ChannelCPUpdaterSuite) TestUpdate() { @@ -53,8 +53,8 @@ func (s *ChannelCPUpdaterSuite) TestUpdate() { return nil }) - go s.updater.start() - defer s.updater.close() + go s.updater.Start() + defer s.updater.Close() tasksNum := 100000 counter := atomic.NewInt64(0) diff --git a/internal/datanode/meta_util.go b/internal/datanode/util/meta_util.go similarity index 95% rename from internal/datanode/meta_util.go rename to internal/datanode/util/meta_util.go index 1066c1d20e491..0dbf6a0bb05b5 100644 --- a/internal/datanode/meta_util.go +++ b/internal/datanode/util/meta_util.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package util import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" @@ -22,8 +22,8 @@ import ( "github.com/milvus-io/milvus/internal/proto/etcdpb" ) -// reviseVChannelInfo will revise the datapb.VchannelInfo for upgrade compatibility from 2.0.2 -func reviseVChannelInfo(vChannel *datapb.VchannelInfo) { +// ReviseVChannelInfo will revise the datapb.VchannelInfo for upgrade compatibility from 2.0.2 +func ReviseVChannelInfo(vChannel *datapb.VchannelInfo) { removeDuplicateSegmentIDFn := func(ids []int64) []int64 { result := make([]int64, 0, len(ids)) existDict := make(map[int64]bool) diff --git a/internal/datanode/rate_collector.go b/internal/datanode/util/rate_collector.go similarity index 62% rename from internal/datanode/rate_collector.go rename to internal/datanode/util/rate_collector.go index 10e76ea5483cd..f7fcd886ae8c9 100644 --- a/internal/datanode/rate_collector.go +++ b/internal/datanode/util/rate_collector.go @@ -14,65 +14,76 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package util import ( "sync" + "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/ratelimitutil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) -// rateCol is global rateCollector in DataNode. +// RateCol is global RateCollector in DataNode. var ( - rateCol *rateCollector + RateCol *RateCollector initOnce sync.Once ) -// rateCollector helps to collect and calculate values (like rate, timeTick and etc...). -type rateCollector struct { +// RateCollector helps to collect and calculate values (like rate, timeTick and etc...). +type RateCollector struct { *ratelimitutil.RateCollector flowGraphTtMu sync.Mutex flowGraphTt map[string]Timestamp } -func initGlobalRateCollector() error { +func InitGlobalRateCollector() error { var err error initOnce.Do(func() { - rateCol, err = newRateCollector() + RateCol, err = NewRateCollector() }) + RateCol.Register(metricsinfo.InsertConsumeThroughput) + RateCol.Register(metricsinfo.DeleteConsumeThroughput) return err } -// newRateCollector returns a new rateCollector. -func newRateCollector() (*rateCollector, error) { +func DeregisterRateCollector(label string) { + RateCol.Deregister(label) +} + +func RegisterRateCollector(label string) { + RateCol.Register(label) +} + +// newRateCollector returns a new RateCollector. +func NewRateCollector() (*RateCollector, error) { rc, err := ratelimitutil.NewRateCollector(ratelimitutil.DefaultWindow, ratelimitutil.DefaultGranularity, false) if err != nil { return nil, err } - return &rateCollector{ + return &RateCollector{ RateCollector: rc, flowGraphTt: make(map[string]Timestamp), }, nil } -// updateFlowGraphTt updates rateCollector's flow graph time tick. -func (r *rateCollector) updateFlowGraphTt(channel string, t Timestamp) { +// UpdateFlowGraphTt updates RateCollector's flow graph time tick. +func (r *RateCollector) UpdateFlowGraphTt(channel string, t Timestamp) { r.flowGraphTtMu.Lock() defer r.flowGraphTtMu.Unlock() r.flowGraphTt[channel] = t } -// removeFlowGraphChannel removes channel from flowGraphTt. -func (r *rateCollector) removeFlowGraphChannel(channel string) { +// RemoveFlowGraphChannel removes channel from flowGraphTt. +func (r *RateCollector) RemoveFlowGraphChannel(channel string) { r.flowGraphTtMu.Lock() defer r.flowGraphTtMu.Unlock() delete(r.flowGraphTt, channel) } -// getMinFlowGraphTt returns the vchannel and minimal time tick of flow graphs. -func (r *rateCollector) getMinFlowGraphTt() (string, Timestamp) { +// GetMinFlowGraphTt returns the vchannel and minimal time tick of flow graphs. +func (r *RateCollector) GetMinFlowGraphTt() (string, Timestamp) { r.flowGraphTtMu.Lock() defer r.flowGraphTtMu.Unlock() minTt := typeutil.MaxTimestamp diff --git a/internal/datanode/rate_collector_test.go b/internal/datanode/util/rate_collector_test.go similarity index 80% rename from internal/datanode/rate_collector_test.go rename to internal/datanode/util/rate_collector_test.go index fa6cc7d201d3d..e5c8dbe4c15c8 100644 --- a/internal/datanode/rate_collector_test.go +++ b/internal/datanode/util/rate_collector_test.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package util import ( "testing" @@ -26,16 +26,16 @@ import ( func TestRateCollector(t *testing.T) { t.Run("test FlowGraphTt", func(t *testing.T) { - collector, err := newRateCollector() + collector, err := NewRateCollector() assert.NoError(t, err) - c, minTt := collector.getMinFlowGraphTt() + c, minTt := collector.GetMinFlowGraphTt() assert.Equal(t, "", c) assert.Equal(t, typeutil.MaxTimestamp, minTt) - collector.updateFlowGraphTt("channel1", 100) - collector.updateFlowGraphTt("channel2", 200) - collector.updateFlowGraphTt("channel3", 50) - c, minTt = collector.getMinFlowGraphTt() + collector.UpdateFlowGraphTt("channel1", 100) + collector.UpdateFlowGraphTt("channel2", 200) + collector.UpdateFlowGraphTt("channel3", 50) + c, minTt = collector.GetMinFlowGraphTt() assert.Equal(t, "channel3", c) assert.Equal(t, Timestamp(50), minTt) }) diff --git a/internal/datanode/mock_test.go b/internal/datanode/util/testutils.go similarity index 79% rename from internal/datanode/mock_test.go rename to internal/datanode/util/testutils.go index 92286c4bbfcb0..f0e6932e93ebe 100644 --- a/internal/datanode/mock_test.go +++ b/internal/datanode/util/testutils.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package util import ( "bytes" @@ -22,12 +22,8 @@ import ( "encoding/binary" "fmt" "math" - "math/rand" - "time" "github.com/cockroachdb/errors" - "github.com/samber/lo" - "github.com/stretchr/testify/mock" "go.uber.org/zap" "google.golang.org/grpc" @@ -35,10 +31,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/datanode/broker" "github.com/milvus-io/milvus/internal/datanode/metacache" - "github.com/milvus-io/milvus/internal/datanode/syncmgr" - "github.com/milvus-io/milvus/internal/datanode/writebuffer" "github.com/milvus-io/milvus/internal/kv" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/proto/datapb" @@ -47,19 +40,22 @@ import ( "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" - "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" "github.com/milvus-io/milvus/pkg/mq/msgstream" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" - "github.com/milvus-io/milvus/pkg/util/tsoutil" - "github.com/milvus-io/milvus/pkg/util/typeutil" ) -const ctxTimeInMillisecond = 5000 +const returnError = "ReturnError" + +type ctxKey struct{} + +const ( + Test = "test" + ctxTimeInMillisecond = 5000 +) // As used in data_sync_service_test.go var segID2SegInfo = map[int64]*datapb.SegmentInfo{ @@ -81,52 +77,31 @@ 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) { +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()) + paramtable.Get().EtcdCfg.UseEmbedEtcd.GetAsBool(), + paramtable.Get().EtcdCfg.EtcdUseSSL.GetAsBool(), + paramtable.Get().EtcdCfg.Endpoints.GetAsStrings(), + paramtable.Get().EtcdCfg.EtcdTLSCert.GetValue(), + paramtable.Get().EtcdCfg.EtcdTLSKey.GetValue(), + paramtable.Get().EtcdCfg.EtcdTLSCACert.GetValue(), + paramtable.Get().EtcdCfg.EtcdTLSMinVersion.GetValue()) if err != nil { return nil, err } - return etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath.GetValue()), nil + return etcdkv.NewEtcdKV(etcdCli, paramtable.Get().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()) + paramtable.Get().EtcdCfg.UseEmbedEtcd.GetAsBool(), + paramtable.Get().EtcdCfg.EtcdUseSSL.GetAsBool(), + paramtable.Get().EtcdCfg.Endpoints.GetAsStrings(), + paramtable.Get().EtcdCfg.EtcdTLSCert.GetValue(), + paramtable.Get().EtcdCfg.EtcdTLSKey.GetValue(), + paramtable.Get().EtcdCfg.EtcdTLSCACert.GetValue(), + paramtable.Get().EtcdCfg.EtcdTLSMinVersion.GetValue()) if err != nil { return err } @@ -877,66 +852,6 @@ func (df *DataFactory) GenMsgStreamDeleteMsgWithTs(idx int, pks []storage.Primar return msg } -func genFlowGraphInsertMsg(chanName string) flowGraphMsg { - timeRange := TimeRange{ - timestampMin: 0, - timestampMax: math.MaxUint64, - } - - startPos := []*msgpb.MsgPosition{ - { - ChannelName: chanName, - MsgID: make([]byte, 0), - Timestamp: tsoutil.ComposeTSByTime(time.Now(), 0), - }, - } - - fgMsg := &flowGraphMsg{ - insertMessages: make([]*msgstream.InsertMsg, 0), - timeRange: TimeRange{ - timestampMin: timeRange.timestampMin, - timestampMax: timeRange.timestampMax, - }, - startPositions: startPos, - endPositions: startPos, - } - - dataFactory := NewDataFactory() - fgMsg.insertMessages = append(fgMsg.insertMessages, dataFactory.GetMsgStreamInsertMsgs(2)...) - - return *fgMsg -} - -func genFlowGraphDeleteMsg(pks []storage.PrimaryKey, chanName string) flowGraphMsg { - timeRange := TimeRange{ - timestampMin: 0, - timestampMax: math.MaxUint64, - } - - startPos := []*msgpb.MsgPosition{ - { - ChannelName: chanName, - MsgID: make([]byte, 0), - Timestamp: 0, - }, - } - - fgMsg := &flowGraphMsg{ - insertMessages: make([]*msgstream.InsertMsg, 0), - timeRange: TimeRange{ - timestampMin: timeRange.timestampMin, - timestampMax: timeRange.timestampMax, - }, - startPositions: startPos, - endPositions: startPos, - } - - dataFactory := NewDataFactory() - fgMsg.deleteMessages = append(fgMsg.deleteMessages, dataFactory.GenMsgStreamDeleteMsg(pks, chanName)) - - return *fgMsg -} - func (m *RootCoordFactory) setCollectionID(id UniqueID) { m.collectionID = id } @@ -1065,139 +980,12 @@ func (f *FailMessageStreamFactory) NewTtMsgStream(ctx context.Context) (msgstrea return nil, errors.New("mocked failure") } -func genInsertDataWithPKs(PKs [2]storage.PrimaryKey, dataType schemapb.DataType) *InsertData { - iD := genInsertData(2) - switch dataType { - case schemapb.DataType_Int64: - values := make([]int64, len(PKs)) - for index, pk := range PKs { - values[index] = pk.(*storage.Int64PrimaryKey).Value - } - iD.Data[106].(*storage.Int64FieldData).Data = values - case schemapb.DataType_VarChar: - values := make([]string, len(PKs)) - for index, pk := range PKs { - values[index] = pk.(*storage.VarCharPrimaryKey).Value - } - iD.Data[109].(*storage.StringFieldData).Data = values - default: - // TODO:: - } - return iD -} - -func genTestStat(meta *etcdpb.CollectionMeta) *storage.PrimaryKeyStats { - var pkFieldID, pkFieldType int64 - for _, field := range meta.Schema.Fields { - if field.IsPrimaryKey { - pkFieldID = field.FieldID - pkFieldType = int64(field.DataType) - } - } - stats, _ := storage.NewPrimaryKeyStats(pkFieldID, pkFieldType, 100) - return stats -} - -func genInsertData(rowNum int) *InsertData { - return &InsertData{ - Data: map[int64]storage.FieldData{ - 0: &storage.Int64FieldData{ - Data: lo.RepeatBy(rowNum, func(i int) int64 { return int64(i + 1) }), - }, - 1: &storage.Int64FieldData{ - Data: lo.RepeatBy(rowNum, func(i int) int64 { return int64(i + 3) }), - }, - 100: &storage.FloatVectorFieldData{ - Data: lo.RepeatBy(rowNum*2, func(i int) float32 { return rand.Float32() }), - Dim: 2, - }, - 101: &storage.BinaryVectorFieldData{ - Data: lo.RepeatBy(rowNum*4, func(i int) byte { return byte(rand.Intn(256)) }), - Dim: 32, - }, - 102: &storage.BoolFieldData{ - Data: lo.RepeatBy(rowNum, func(i int) bool { return i%2 == 0 }), - }, - 103: &storage.Int8FieldData{ - Data: lo.RepeatBy(rowNum, func(i int) int8 { return int8(i) }), - }, - 104: &storage.Int16FieldData{ - Data: lo.RepeatBy(rowNum, func(i int) int16 { return int16(i) }), - }, - 105: &storage.Int32FieldData{ - Data: lo.RepeatBy(rowNum, func(i int) int32 { return int32(i) }), - }, - 106: &storage.Int64FieldData{ - Data: lo.RepeatBy(rowNum, func(i int) int64 { return int64(i) }), - }, - 107: &storage.FloatFieldData{ - Data: lo.RepeatBy(rowNum, func(i int) float32 { return rand.Float32() }), - }, - 108: &storage.DoubleFieldData{ - Data: lo.RepeatBy(rowNum, func(i int) float64 { return rand.Float64() }), - }, - 109: &storage.StringFieldData{ - Data: lo.RepeatBy(rowNum, func(i int) string { return fmt.Sprintf("test%d", i) }), - }, - }, - } -} - -func genEmptyInsertData() *InsertData { - return &InsertData{ - Data: map[int64]storage.FieldData{ - 0: &storage.Int64FieldData{ - Data: []int64{}, - }, - 1: &storage.Int64FieldData{ - Data: []int64{}, - }, - 100: &storage.FloatVectorFieldData{ - Data: []float32{}, - Dim: 2, - }, - 101: &storage.BinaryVectorFieldData{ - Data: []byte{}, - Dim: 32, - }, - 102: &storage.BoolFieldData{ - Data: []bool{}, - }, - 103: &storage.Int8FieldData{ - Data: []int8{}, - }, - 104: &storage.Int16FieldData{ - Data: []int16{}, - }, - 105: &storage.Int32FieldData{ - Data: []int32{}, - }, - 106: &storage.Int64FieldData{ - Data: []int64{}, - }, - 107: &storage.FloatFieldData{ - Data: []float32{}, - }, - 108: &storage.DoubleFieldData{ - Data: []float64{}, - }, - 109: &storage.StringFieldData{ - Data: []string{}, - }, - }, - } -} - -func genTestTickler() *etcdTickler { - return newEtcdTickler(0, "", nil, nil, 0) -} - // MockDataSuiteBase compose some mock dependency to generate test dataset type MockDataSuiteBase struct { schema *schemapb.CollectionSchema } -func (s *MockDataSuiteBase) prepareData() { +func (s *MockDataSuiteBase) PrepareData() { s.schema = &schemapb.CollectionSchema{ Name: "test_collection", Fields: []*schemapb.FieldSchema{ @@ -1214,3 +1002,34 @@ func (s *MockDataSuiteBase) prepareData() { func EmptyBfsFactory(info *datapb.SegmentInfo) *metacache.BloomFilterSet { return metacache.NewBloomFilterSet() } + +func GetWatchInfoByOpID(opID UniqueID, channel string, state datapb.ChannelWatchState) *datapb.ChannelWatchInfo { + return &datapb.ChannelWatchInfo{ + OpID: opID, + State: state, + Vchan: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: channel, + }, + Schema: &schemapb.CollectionSchema{ + Name: "test_collection", + Fields: []*schemapb.FieldSchema{ + { + FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64, + }, + { + FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64, + }, + { + FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true, + }, + { + FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector, + TypeParams: []*commonpb.KeyValuePair{ + {Key: common.DimKey, Value: "128"}, + }, + }, + }, + }, + } +} diff --git a/internal/datanode/util/tickler.go b/internal/datanode/util/tickler.go new file mode 100644 index 0000000000000..04d3ba1bf129b --- /dev/null +++ b/internal/datanode/util/tickler.go @@ -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), + } +} diff --git a/internal/datanode/timetick_sender.go b/internal/datanode/util/timetick_sender.go similarity index 80% rename from internal/datanode/timetick_sender.go rename to internal/datanode/util/timetick_sender.go index ecce410c05501..233fa7c02afb1 100644 --- a/internal/datanode/timetick_sender.go +++ b/internal/datanode/util/timetick_sender.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package util import ( "context" @@ -29,13 +29,14 @@ import ( "github.com/milvus-io/milvus/internal/datanode/broker" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/commonpbutil" + "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/retry" ) -// timeTickSender is to merge channel states updated by flow graph node and send to datacoord periodically -// timeTickSender hold segmentStats cache for each channel, +// TimeTickSender is to merge channel states updated by flow graph node and send to datacoord periodically +// TimeTickSender hold segmentStats cache for each channel, // after send succeeds will clean the cache earlier than last sent timestamp -type timeTickSender struct { +type TimeTickSender struct { nodeID int64 broker broker.Broker @@ -53,14 +54,14 @@ type channelStats struct { lastTs uint64 } -// data struct only used in timeTickSender +// data struct only used in TimeTickSender type segmentStats struct { *commonpb.SegmentStats ts uint64 } -func newTimeTickSender(broker broker.Broker, nodeID int64, opts ...retry.Option) *timeTickSender { - return &timeTickSender{ +func NewTimeTickSender(broker broker.Broker, nodeID int64, opts ...retry.Option) *TimeTickSender { + return &TimeTickSender{ nodeID: nodeID, broker: broker, statsCache: make(map[string]*channelStats), @@ -68,7 +69,7 @@ func newTimeTickSender(broker broker.Broker, nodeID int64, opts ...retry.Option) } } -func (m *timeTickSender) start() { +func (m *TimeTickSender) Start() { m.wg.Add(1) ctx, cancel := context.WithCancel(context.Background()) m.cancelFunc = cancel @@ -78,20 +79,20 @@ func (m *timeTickSender) start() { }() } -func (m *timeTickSender) Stop() { +func (m *TimeTickSender) Stop() { if m.cancelFunc != nil { m.cancelFunc() m.wg.Wait() } } -func (m *timeTickSender) work(ctx context.Context) { - ticker := time.NewTicker(Params.DataNodeCfg.DataNodeTimeTickInterval.GetAsDuration(time.Millisecond)) +func (m *TimeTickSender) work(ctx context.Context) { + ticker := time.NewTicker(paramtable.Get().DataNodeCfg.DataNodeTimeTickInterval.GetAsDuration(time.Millisecond)) defer ticker.Stop() for { select { case <-ctx.Done(): - log.Info("timeTickSender context done") + log.Info("TimeTickSender context done") return case <-ticker.C: m.sendReport(ctx) @@ -99,7 +100,7 @@ func (m *timeTickSender) work(ctx context.Context) { } } -func (m *timeTickSender) update(channelName string, timestamp uint64, segStats []*commonpb.SegmentStats) { +func (m *TimeTickSender) Update(channelName string, timestamp uint64, segStats []*commonpb.SegmentStats) { m.mu.Lock() defer m.mu.Unlock() _, ok := m.statsCache[channelName] @@ -118,7 +119,7 @@ func (m *timeTickSender) update(channelName string, timestamp uint64, segStats [ m.statsCache[channelName].lastTs = timestamp } -func (m *timeTickSender) assembleDatanodeTtMsg() ([]*msgpb.DataNodeTtMsg, map[string]uint64) { +func (m *TimeTickSender) assembleDatanodeTtMsg() ([]*msgpb.DataNodeTtMsg, map[string]uint64) { m.mu.RLock() defer m.mu.RUnlock() @@ -144,7 +145,7 @@ func (m *timeTickSender) assembleDatanodeTtMsg() ([]*msgpb.DataNodeTtMsg, map[st return msgs, lastSentTss } -func (m *timeTickSender) cleanStatesCache(lastSentTss map[string]uint64) { +func (m *TimeTickSender) cleanStatesCache(lastSentTss map[string]uint64) { m.mu.Lock() defer m.mu.Unlock() sizeBeforeClean := len(m.statsCache) @@ -161,12 +162,12 @@ func (m *timeTickSender) cleanStatesCache(lastSentTss map[string]uint64) { delete(m.statsCache, channelName) } } - log.RatedDebug(30, "timeTickSender stats", zap.Any("lastSentTss", lastSentTss), zap.Int("sizeBeforeClean", sizeBeforeClean), zap.Int("sizeAfterClean", len(m.statsCache))) + log.RatedDebug(30, "TimeTickSender stats", zap.Any("lastSentTss", lastSentTss), zap.Int("sizeBeforeClean", sizeBeforeClean), zap.Int("sizeAfterClean", len(m.statsCache))) } -func (m *timeTickSender) sendReport(ctx context.Context) error { +func (m *TimeTickSender) sendReport(ctx context.Context) error { toSendMsgs, sendLastTss := m.assembleDatanodeTtMsg() - log.RatedDebug(30, "timeTickSender send datanode timetick message", zap.Any("toSendMsgs", toSendMsgs), zap.Any("sendLastTss", sendLastTss)) + log.RatedDebug(30, "TimeTickSender send datanode timetick message", zap.Any("toSendMsgs", toSendMsgs), zap.Any("sendLastTss", sendLastTss)) err := retry.Do(ctx, func() error { return m.broker.ReportTimeTick(ctx, toSendMsgs) }, m.options...) diff --git a/internal/datanode/timetick_sender_test.go b/internal/datanode/util/timetick_sender_test.go similarity index 96% rename from internal/datanode/timetick_sender_test.go rename to internal/datanode/util/timetick_sender_test.go index 61e7245a27232..68f2bed790bd0 100644 --- a/internal/datanode/timetick_sender_test.go +++ b/internal/datanode/util/timetick_sender_test.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package util import ( "context" @@ -40,7 +40,7 @@ func TestTimetickManagerNormal(t *testing.T) { broker := broker.NewMockBroker(t) broker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(nil).Maybe() - manager := newTimeTickSender(broker, 0) + manager := NewTimeTickSender(broker, 0) channelName1 := "channel1" ts := uint64(time.Now().UnixMilli()) @@ -145,7 +145,7 @@ func TestTimetickManagerSendErr(t *testing.T) { broker := broker.NewMockBroker(t) broker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(errors.New("mock")).Maybe() - manager := newTimeTickSender(broker, 0, retry.Attempts(1)) + manager := NewTimeTickSender(broker, 0, retry.Attempts(1)) channelName1 := "channel1" ts := uint64(time.Now().Unix()) @@ -174,8 +174,8 @@ func TestTimetickManagerSendReport(t *testing.T) { }). Return(nil) mockDataCoord.EXPECT().ReportDataNodeTtMsgs(mock.Anything, mock.Anything).Return(merr.Status(nil), nil).Maybe() - manager := newTimeTickSender(broker, 0) - manager.start() + manager := NewTimeTickSender(broker, 0) + manager.Start() assert.Eventually(t, func() bool { return called.Load() diff --git a/internal/datanode/util.go b/internal/datanode/util/util.go similarity index 57% rename from internal/datanode/util.go rename to internal/datanode/util/util.go index dc1fbdab2c278..e4925e537fa22 100644 --- a/internal/datanode/util.go +++ b/internal/datanode/util/util.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package util import ( "context" @@ -22,6 +22,15 @@ import ( "go.opentelemetry.io/otel" "go.opentelemetry.io/otel/trace" + "github.com/milvus-io/milvus/internal/datanode/allocator" + "github.com/milvus-io/milvus/internal/datanode/broker" + "github.com/milvus-io/milvus/internal/datanode/compaction" + "github.com/milvus-io/milvus/internal/datanode/syncmgr" + "github.com/milvus-io/milvus/internal/datanode/writebuffer" + "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/internal/util/dependency" + "github.com/milvus-io/milvus/internal/util/sessionutil" + "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" "github.com/milvus-io/milvus/pkg/mq/msgstream" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -40,13 +49,28 @@ type ( DSL = string ) +type PipelineParams struct { + Ctx context.Context + Broker broker.Broker + SyncMgr syncmgr.SyncManager + TimeTickSender *TimeTickSender // reference to TimeTickSender + CompactionExecutor compaction.Executor // reference to compaction executor + MsgStreamFactory dependency.Factory + DispClient msgdispatcher.Client + ChunkManager storage.ChunkManager + Session *sessionutil.Session + WriteBufferManager writebuffer.BufferManager + CheckpointUpdater *ChannelCheckpointUpdater + Allocator allocator.Allocator +} + // TimeRange is a range of timestamp contains the min-timestamp and max-timestamp type TimeRange struct { - timestampMin Timestamp - timestampMax Timestamp + TimestampMin Timestamp + TimestampMax Timestamp } -func startTracer(msg msgstream.TsMsg, name string) (context.Context, trace.Span) { +func StartTracer(msg msgstream.TsMsg, name string) (context.Context, trace.Span) { ctx := msg.TraceCtx() if ctx == nil { ctx = context.Background() diff --git a/internal/mocks/mock_datanode.go b/internal/mocks/mock_datanode.go index 6ba06b0009b15..190da75be0885 100644 --- a/internal/mocks/mock_datanode.go +++ b/internal/mocks/mock_datanode.go @@ -64,8 +64,8 @@ type MockDataNode_CheckChannelOperationProgress_Call struct { } // CheckChannelOperationProgress is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.ChannelWatchInfo +// - _a0 context.Context +// - _a1 *datapb.ChannelWatchInfo func (_e *MockDataNode_Expecter) CheckChannelOperationProgress(_a0 interface{}, _a1 interface{}) *MockDataNode_CheckChannelOperationProgress_Call { return &MockDataNode_CheckChannelOperationProgress_Call{Call: _e.mock.On("CheckChannelOperationProgress", _a0, _a1)} } @@ -119,8 +119,8 @@ type MockDataNode_CompactionV2_Call struct { } // CompactionV2 is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.CompactionPlan +// - _a0 context.Context +// - _a1 *datapb.CompactionPlan func (_e *MockDataNode_Expecter) CompactionV2(_a0 interface{}, _a1 interface{}) *MockDataNode_CompactionV2_Call { return &MockDataNode_CompactionV2_Call{Call: _e.mock.On("CompactionV2", _a0, _a1)} } @@ -174,8 +174,8 @@ type MockDataNode_DropCompactionPlan_Call struct { } // DropCompactionPlan is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.DropCompactionPlanRequest +// - _a0 context.Context +// - _a1 *datapb.DropCompactionPlanRequest func (_e *MockDataNode_Expecter) DropCompactionPlan(_a0 interface{}, _a1 interface{}) *MockDataNode_DropCompactionPlan_Call { return &MockDataNode_DropCompactionPlan_Call{Call: _e.mock.On("DropCompactionPlan", _a0, _a1)} } @@ -229,8 +229,8 @@ type MockDataNode_DropImport_Call struct { } // DropImport is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.DropImportRequest +// - _a0 context.Context +// - _a1 *datapb.DropImportRequest func (_e *MockDataNode_Expecter) DropImport(_a0 interface{}, _a1 interface{}) *MockDataNode_DropImport_Call { return &MockDataNode_DropImport_Call{Call: _e.mock.On("DropImport", _a0, _a1)} } @@ -284,8 +284,8 @@ type MockDataNode_FlushChannels_Call struct { } // FlushChannels is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.FlushChannelsRequest +// - _a0 context.Context +// - _a1 *datapb.FlushChannelsRequest func (_e *MockDataNode_Expecter) FlushChannels(_a0 interface{}, _a1 interface{}) *MockDataNode_FlushChannels_Call { return &MockDataNode_FlushChannels_Call{Call: _e.mock.On("FlushChannels", _a0, _a1)} } @@ -339,8 +339,8 @@ type MockDataNode_FlushSegments_Call struct { } // FlushSegments is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.FlushSegmentsRequest +// - _a0 context.Context +// - _a1 *datapb.FlushSegmentsRequest func (_e *MockDataNode_Expecter) FlushSegments(_a0 interface{}, _a1 interface{}) *MockDataNode_FlushSegments_Call { return &MockDataNode_FlushSegments_Call{Call: _e.mock.On("FlushSegments", _a0, _a1)} } @@ -435,8 +435,8 @@ type MockDataNode_GetCompactionState_Call struct { } // GetCompactionState is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.CompactionStateRequest +// - _a0 context.Context +// - _a1 *datapb.CompactionStateRequest func (_e *MockDataNode_Expecter) GetCompactionState(_a0 interface{}, _a1 interface{}) *MockDataNode_GetCompactionState_Call { return &MockDataNode_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState", _a0, _a1)} } @@ -490,8 +490,8 @@ type MockDataNode_GetComponentStates_Call struct { } // GetComponentStates is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *milvuspb.GetComponentStatesRequest +// - _a0 context.Context +// - _a1 *milvuspb.GetComponentStatesRequest func (_e *MockDataNode_Expecter) GetComponentStates(_a0 interface{}, _a1 interface{}) *MockDataNode_GetComponentStates_Call { return &MockDataNode_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", _a0, _a1)} } @@ -545,8 +545,8 @@ type MockDataNode_GetMetrics_Call struct { } // GetMetrics is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *milvuspb.GetMetricsRequest +// - _a0 context.Context +// - _a1 *milvuspb.GetMetricsRequest func (_e *MockDataNode_Expecter) GetMetrics(_a0 interface{}, _a1 interface{}) *MockDataNode_GetMetrics_Call { return &MockDataNode_GetMetrics_Call{Call: _e.mock.On("GetMetrics", _a0, _a1)} } @@ -682,8 +682,8 @@ type MockDataNode_GetStatisticsChannel_Call struct { } // GetStatisticsChannel is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *internalpb.GetStatisticsChannelRequest +// - _a0 context.Context +// - _a1 *internalpb.GetStatisticsChannelRequest func (_e *MockDataNode_Expecter) GetStatisticsChannel(_a0 interface{}, _a1 interface{}) *MockDataNode_GetStatisticsChannel_Call { return &MockDataNode_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", _a0, _a1)} } @@ -737,8 +737,8 @@ type MockDataNode_ImportV2_Call struct { } // ImportV2 is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.ImportRequest +// - _a0 context.Context +// - _a1 *datapb.ImportRequest func (_e *MockDataNode_Expecter) ImportV2(_a0 interface{}, _a1 interface{}) *MockDataNode_ImportV2_Call { return &MockDataNode_ImportV2_Call{Call: _e.mock.On("ImportV2", _a0, _a1)} } @@ -833,8 +833,8 @@ type MockDataNode_NotifyChannelOperation_Call struct { } // NotifyChannelOperation is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.ChannelOperationsRequest +// - _a0 context.Context +// - _a1 *datapb.ChannelOperationsRequest func (_e *MockDataNode_Expecter) NotifyChannelOperation(_a0 interface{}, _a1 interface{}) *MockDataNode_NotifyChannelOperation_Call { return &MockDataNode_NotifyChannelOperation_Call{Call: _e.mock.On("NotifyChannelOperation", _a0, _a1)} } @@ -888,8 +888,8 @@ type MockDataNode_PreImport_Call struct { } // PreImport is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.PreImportRequest +// - _a0 context.Context +// - _a1 *datapb.PreImportRequest func (_e *MockDataNode_Expecter) PreImport(_a0 interface{}, _a1 interface{}) *MockDataNode_PreImport_Call { return &MockDataNode_PreImport_Call{Call: _e.mock.On("PreImport", _a0, _a1)} } @@ -943,8 +943,8 @@ type MockDataNode_QueryImport_Call struct { } // QueryImport is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.QueryImportRequest +// - _a0 context.Context +// - _a1 *datapb.QueryImportRequest func (_e *MockDataNode_Expecter) QueryImport(_a0 interface{}, _a1 interface{}) *MockDataNode_QueryImport_Call { return &MockDataNode_QueryImport_Call{Call: _e.mock.On("QueryImport", _a0, _a1)} } @@ -998,8 +998,8 @@ type MockDataNode_QueryPreImport_Call struct { } // QueryPreImport is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.QueryPreImportRequest +// - _a0 context.Context +// - _a1 *datapb.QueryPreImportRequest func (_e *MockDataNode_Expecter) QueryPreImport(_a0 interface{}, _a1 interface{}) *MockDataNode_QueryPreImport_Call { return &MockDataNode_QueryPreImport_Call{Call: _e.mock.On("QueryPreImport", _a0, _a1)} } @@ -1053,8 +1053,8 @@ type MockDataNode_QuerySlot_Call struct { } // QuerySlot is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.QuerySlotRequest +// - _a0 context.Context +// - _a1 *datapb.QuerySlotRequest func (_e *MockDataNode_Expecter) QuerySlot(_a0 interface{}, _a1 interface{}) *MockDataNode_QuerySlot_Call { return &MockDataNode_QuerySlot_Call{Call: _e.mock.On("QuerySlot", _a0, _a1)} } @@ -1149,8 +1149,8 @@ type MockDataNode_ResendSegmentStats_Call struct { } // ResendSegmentStats is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.ResendSegmentStatsRequest +// - _a0 context.Context +// - _a1 *datapb.ResendSegmentStatsRequest func (_e *MockDataNode_Expecter) ResendSegmentStats(_a0 interface{}, _a1 interface{}) *MockDataNode_ResendSegmentStats_Call { return &MockDataNode_ResendSegmentStats_Call{Call: _e.mock.On("ResendSegmentStats", _a0, _a1)} } @@ -1183,7 +1183,7 @@ type MockDataNode_SetAddress_Call struct { } // SetAddress is a helper method to define mock.On call -// - address string +// - address string func (_e *MockDataNode_Expecter) SetAddress(address interface{}) *MockDataNode_SetAddress_Call { return &MockDataNode_SetAddress_Call{Call: _e.mock.On("SetAddress", address)} } @@ -1225,7 +1225,7 @@ type MockDataNode_SetDataCoordClient_Call struct { } // SetDataCoordClient is a helper method to define mock.On call -// - dataCoord types.DataCoordClient +// - dataCoord types.DataCoordClient func (_e *MockDataNode_Expecter) SetDataCoordClient(dataCoord interface{}) *MockDataNode_SetDataCoordClient_Call { return &MockDataNode_SetDataCoordClient_Call{Call: _e.mock.On("SetDataCoordClient", dataCoord)} } @@ -1258,7 +1258,7 @@ type MockDataNode_SetEtcdClient_Call struct { } // SetEtcdClient is a helper method to define mock.On call -// - etcdClient *clientv3.Client +// - etcdClient *clientv3.Client func (_e *MockDataNode_Expecter) SetEtcdClient(etcdClient interface{}) *MockDataNode_SetEtcdClient_Call { return &MockDataNode_SetEtcdClient_Call{Call: _e.mock.On("SetEtcdClient", etcdClient)} } @@ -1300,7 +1300,7 @@ type MockDataNode_SetRootCoordClient_Call struct { } // SetRootCoordClient is a helper method to define mock.On call -// - rootCoord types.RootCoordClient +// - rootCoord types.RootCoordClient func (_e *MockDataNode_Expecter) SetRootCoordClient(rootCoord interface{}) *MockDataNode_SetRootCoordClient_Call { return &MockDataNode_SetRootCoordClient_Call{Call: _e.mock.On("SetRootCoordClient", rootCoord)} } @@ -1354,8 +1354,8 @@ type MockDataNode_ShowConfigurations_Call struct { } // ShowConfigurations is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *internalpb.ShowConfigurationsRequest +// - _a0 context.Context +// - _a1 *internalpb.ShowConfigurationsRequest func (_e *MockDataNode_Expecter) ShowConfigurations(_a0 interface{}, _a1 interface{}) *MockDataNode_ShowConfigurations_Call { return &MockDataNode_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", _a0, _a1)} } @@ -1491,8 +1491,8 @@ type MockDataNode_SyncSegments_Call struct { } // SyncSegments is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.SyncSegmentsRequest +// - _a0 context.Context +// - _a1 *datapb.SyncSegmentsRequest func (_e *MockDataNode_Expecter) SyncSegments(_a0 interface{}, _a1 interface{}) *MockDataNode_SyncSegments_Call { return &MockDataNode_SyncSegments_Call{Call: _e.mock.On("SyncSegments", _a0, _a1)} } @@ -1525,7 +1525,7 @@ type MockDataNode_UpdateStateCode_Call struct { } // UpdateStateCode is a helper method to define mock.On call -// - stateCode commonpb.StateCode +// - stateCode commonpb.StateCode func (_e *MockDataNode_Expecter) UpdateStateCode(stateCode interface{}) *MockDataNode_UpdateStateCode_Call { return &MockDataNode_UpdateStateCode_Call{Call: _e.mock.On("UpdateStateCode", stateCode)} } @@ -1579,8 +1579,8 @@ type MockDataNode_WatchDmChannels_Call struct { } // WatchDmChannels is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.WatchDmChannelsRequest +// - _a0 context.Context +// - _a1 *datapb.WatchDmChannelsRequest func (_e *MockDataNode_Expecter) WatchDmChannels(_a0 interface{}, _a1 interface{}) *MockDataNode_WatchDmChannels_Call { return &MockDataNode_WatchDmChannels_Call{Call: _e.mock.On("WatchDmChannels", _a0, _a1)} } diff --git a/internal/mocks/mock_datanode_client.go b/internal/mocks/mock_datanode_client.go index 0b4f876803145..91661051c390b 100644 --- a/internal/mocks/mock_datanode_client.go +++ b/internal/mocks/mock_datanode_client.go @@ -70,9 +70,9 @@ type MockDataNodeClient_CheckChannelOperationProgress_Call struct { } // CheckChannelOperationProgress is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.ChannelWatchInfo -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.ChannelWatchInfo +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) CheckChannelOperationProgress(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_CheckChannelOperationProgress_Call { return &MockDataNodeClient_CheckChannelOperationProgress_Call{Call: _e.mock.On("CheckChannelOperationProgress", append([]interface{}{ctx, in}, opts...)...)} @@ -181,9 +181,9 @@ type MockDataNodeClient_CompactionV2_Call struct { } // CompactionV2 is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.CompactionPlan -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.CompactionPlan +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) CompactionV2(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_CompactionV2_Call { return &MockDataNodeClient_CompactionV2_Call{Call: _e.mock.On("CompactionV2", append([]interface{}{ctx, in}, opts...)...)} @@ -251,9 +251,9 @@ type MockDataNodeClient_DropCompactionPlan_Call struct { } // DropCompactionPlan is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.DropCompactionPlanRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.DropCompactionPlanRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) DropCompactionPlan(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_DropCompactionPlan_Call { return &MockDataNodeClient_DropCompactionPlan_Call{Call: _e.mock.On("DropCompactionPlan", append([]interface{}{ctx, in}, opts...)...)} @@ -321,9 +321,9 @@ type MockDataNodeClient_DropImport_Call struct { } // DropImport is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.DropImportRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.DropImportRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) DropImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_DropImport_Call { return &MockDataNodeClient_DropImport_Call{Call: _e.mock.On("DropImport", append([]interface{}{ctx, in}, opts...)...)} @@ -391,9 +391,9 @@ type MockDataNodeClient_FlushChannels_Call struct { } // FlushChannels is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.FlushChannelsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.FlushChannelsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) FlushChannels(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_FlushChannels_Call { return &MockDataNodeClient_FlushChannels_Call{Call: _e.mock.On("FlushChannels", append([]interface{}{ctx, in}, opts...)...)} @@ -461,9 +461,9 @@ type MockDataNodeClient_FlushSegments_Call struct { } // FlushSegments is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.FlushSegmentsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.FlushSegmentsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) FlushSegments(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_FlushSegments_Call { return &MockDataNodeClient_FlushSegments_Call{Call: _e.mock.On("FlushSegments", append([]interface{}{ctx, in}, opts...)...)} @@ -531,9 +531,9 @@ type MockDataNodeClient_GetCompactionState_Call struct { } // GetCompactionState is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.CompactionStateRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.CompactionStateRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) GetCompactionState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetCompactionState_Call { return &MockDataNodeClient_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState", append([]interface{}{ctx, in}, opts...)...)} @@ -601,9 +601,9 @@ type MockDataNodeClient_GetComponentStates_Call struct { } // GetComponentStates is a helper method to define mock.On call -// - ctx context.Context -// - in *milvuspb.GetComponentStatesRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *milvuspb.GetComponentStatesRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) GetComponentStates(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetComponentStates_Call { return &MockDataNodeClient_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", append([]interface{}{ctx, in}, opts...)...)} @@ -671,9 +671,9 @@ type MockDataNodeClient_GetMetrics_Call struct { } // GetMetrics is a helper method to define mock.On call -// - ctx context.Context -// - in *milvuspb.GetMetricsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *milvuspb.GetMetricsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) GetMetrics(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetMetrics_Call { return &MockDataNodeClient_GetMetrics_Call{Call: _e.mock.On("GetMetrics", append([]interface{}{ctx, in}, opts...)...)} @@ -741,9 +741,9 @@ type MockDataNodeClient_GetStatisticsChannel_Call struct { } // GetStatisticsChannel is a helper method to define mock.On call -// - ctx context.Context -// - in *internalpb.GetStatisticsChannelRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *internalpb.GetStatisticsChannelRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) GetStatisticsChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetStatisticsChannel_Call { return &MockDataNodeClient_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", append([]interface{}{ctx, in}, opts...)...)} @@ -811,9 +811,9 @@ type MockDataNodeClient_ImportV2_Call struct { } // ImportV2 is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.ImportRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.ImportRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) ImportV2(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_ImportV2_Call { return &MockDataNodeClient_ImportV2_Call{Call: _e.mock.On("ImportV2", append([]interface{}{ctx, in}, opts...)...)} @@ -881,9 +881,9 @@ type MockDataNodeClient_NotifyChannelOperation_Call struct { } // NotifyChannelOperation is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.ChannelOperationsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.ChannelOperationsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) NotifyChannelOperation(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_NotifyChannelOperation_Call { return &MockDataNodeClient_NotifyChannelOperation_Call{Call: _e.mock.On("NotifyChannelOperation", append([]interface{}{ctx, in}, opts...)...)} @@ -951,9 +951,9 @@ type MockDataNodeClient_PreImport_Call struct { } // PreImport is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.PreImportRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.PreImportRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) PreImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_PreImport_Call { return &MockDataNodeClient_PreImport_Call{Call: _e.mock.On("PreImport", append([]interface{}{ctx, in}, opts...)...)} @@ -1021,9 +1021,9 @@ type MockDataNodeClient_QueryImport_Call struct { } // QueryImport is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.QueryImportRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.QueryImportRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) QueryImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_QueryImport_Call { return &MockDataNodeClient_QueryImport_Call{Call: _e.mock.On("QueryImport", append([]interface{}{ctx, in}, opts...)...)} @@ -1091,9 +1091,9 @@ type MockDataNodeClient_QueryPreImport_Call struct { } // QueryPreImport is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.QueryPreImportRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.QueryPreImportRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) QueryPreImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_QueryPreImport_Call { return &MockDataNodeClient_QueryPreImport_Call{Call: _e.mock.On("QueryPreImport", append([]interface{}{ctx, in}, opts...)...)} @@ -1161,9 +1161,9 @@ type MockDataNodeClient_QuerySlot_Call struct { } // QuerySlot is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.QuerySlotRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.QuerySlotRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) QuerySlot(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_QuerySlot_Call { return &MockDataNodeClient_QuerySlot_Call{Call: _e.mock.On("QuerySlot", append([]interface{}{ctx, in}, opts...)...)} @@ -1231,9 +1231,9 @@ type MockDataNodeClient_ResendSegmentStats_Call struct { } // ResendSegmentStats is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.ResendSegmentStatsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.ResendSegmentStatsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) ResendSegmentStats(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_ResendSegmentStats_Call { return &MockDataNodeClient_ResendSegmentStats_Call{Call: _e.mock.On("ResendSegmentStats", append([]interface{}{ctx, in}, opts...)...)} @@ -1301,9 +1301,9 @@ type MockDataNodeClient_ShowConfigurations_Call struct { } // ShowConfigurations is a helper method to define mock.On call -// - ctx context.Context -// - in *internalpb.ShowConfigurationsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *internalpb.ShowConfigurationsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) ShowConfigurations(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_ShowConfigurations_Call { return &MockDataNodeClient_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", append([]interface{}{ctx, in}, opts...)...)} @@ -1371,9 +1371,9 @@ type MockDataNodeClient_SyncSegments_Call struct { } // SyncSegments is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.SyncSegmentsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.SyncSegmentsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) SyncSegments(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_SyncSegments_Call { return &MockDataNodeClient_SyncSegments_Call{Call: _e.mock.On("SyncSegments", append([]interface{}{ctx, in}, opts...)...)} @@ -1441,9 +1441,9 @@ type MockDataNodeClient_WatchDmChannels_Call struct { } // WatchDmChannels is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.WatchDmChannelsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.WatchDmChannelsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) WatchDmChannels(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_WatchDmChannels_Call { return &MockDataNodeClient_WatchDmChannels_Call{Call: _e.mock.On("WatchDmChannels", append([]interface{}{ctx, in}, opts...)...)} diff --git a/pkg/util/paramtable/component_param.go b/pkg/util/paramtable/component_param.go index 3376766788897..e43d54ea22db6 100644 --- a/pkg/util/paramtable/component_param.go +++ b/pkg/util/paramtable/component_param.go @@ -2824,7 +2824,6 @@ user-task-polling: type dataCoordConfig struct { // --- CHANNEL --- WatchTimeoutInterval ParamItem `refreshable:"false"` - EnableBalanceChannelWithRPC ParamItem `refreshable:"false"` LegacyVersionWithoutRPCWatch ParamItem `refreshable:"false"` ChannelBalanceSilentDuration ParamItem `refreshable:"true"` ChannelBalanceInterval ParamItem `refreshable:"true"` @@ -2939,15 +2938,6 @@ func (p *dataCoordConfig) init(base *BaseTable) { } p.WatchTimeoutInterval.Init(base.mgr) - p.EnableBalanceChannelWithRPC = ParamItem{ - Key: "dataCoord.channel.balanceWithRpc", - Version: "2.4.0", - DefaultValue: "true", - Doc: "Whether to enable balance with RPC, default to use etcd watch", - Export: true, - } - p.EnableBalanceChannelWithRPC.Init(base.mgr) - p.LegacyVersionWithoutRPCWatch = ParamItem{ Key: "dataCoord.channel.legacyVersionWithoutRPCWatch", Version: "2.4.0", diff --git a/tests/integration/watchcompatibility/watch_test.go b/tests/integration/watchcompatibility/watch_test.go deleted file mode 100644 index 0a0fc3674ff96..0000000000000 --- a/tests/integration/watchcompatibility/watch_test.go +++ /dev/null @@ -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)) -}