From a4439cc911fd1c29b98dcba72f35fd07f6c83182 Mon Sep 17 00:00:00 2001 From: "yihao.dai" Date: Fri, 2 Aug 2024 18:30:23 +0800 Subject: [PATCH] enhance: Implement flusher in streamingNode (#34942) - Implement flusher to: - Manage the pipelines (creation, deletion, etc.) - Manage the segment write buffer - Manage sync operation (including receive flushMsg and execute flush) - Add a new `GetChannelRecoveryInfo` RPC in DataCoord. - Reorganize packages: `flushcommon` and `datanode`. issue: https://github.com/milvus-io/milvus/issues/33285 --------- Signed-off-by: bigsheeper --- Makefile | 22 +- internal/.mockery.yaml | 3 + internal/datacoord/services.go | 36 ++ internal/datacoord/services_test.go | 50 +++ internal/datanode/channel/channel_manager.go | 12 +- .../datanode/channel/channel_manager_test.go | 70 +++- .../compaction/clustering_compactor.go | 2 +- .../compaction/clustering_compactor_test.go | 2 +- .../datanode/compaction/compactor_common.go | 2 +- internal/datanode/compaction/l0_compactor.go | 2 +- .../datanode/compaction/l0_compactor_test.go | 2 +- internal/datanode/compaction/mix_compactor.go | 2 +- .../datanode/compaction/mix_compactor_test.go | 2 +- internal/datanode/data_node.go | 30 +- internal/datanode/data_node_test.go | 23 +- internal/datanode/metrics_info.go | 6 +- internal/datanode/services.go | 2 +- internal/datanode/services_test.go | 4 +- internal/datanode/util/cache.go | 12 +- internal/datanode/util/cache_test.go | 13 +- .../distributed/datacoord/client/client.go | 12 + .../datacoord/client/client_test.go | 47 +++ internal/distributed/datacoord/service.go | 5 + .../distributed/datacoord/service_test.go | 7 + .../broker/broker.go | 0 .../broker/datacoord.go | 0 .../broker/datacoord_test.go | 0 .../broker/mock_broker.go | 0 .../{datanode => flushcommon}/io/binlog_io.go | 0 .../io/binlog_io_test.go | 0 .../{datanode => flushcommon}/io/io_pool.go | 0 .../io/io_pool_test.go | 0 .../io/mock_binlogio.go | 0 .../flushcommon/pipeline/data_sync_service.go | 67 +++- .../pipeline/data_sync_service_test.go | 52 ++- .../pipeline/flow_graph_dd_node.go | 31 +- .../pipeline/flow_graph_dd_node_test.go | 99 ++--- .../flow_graph_dmstream_input_node.go | 33 +- .../flow_graph_dmstream_input_node_test.go | 2 +- .../pipeline/flow_graph_manager.go | 6 +- .../pipeline/flow_graph_manager_test.go | 8 +- .../pipeline/flow_graph_message.go | 9 +- .../pipeline/flow_graph_message_test.go | 5 +- .../pipeline/flow_graph_time_tick_node.go | 2 +- .../pipeline/flow_graph_write_node.go | 2 +- .../pipeline/testutils_test.go} | 49 +-- internal/flushcommon/syncmgr/meta_writer.go | 2 +- .../flushcommon/syncmgr/meta_writer_test.go | 2 +- internal/flushcommon/syncmgr/sync_manager.go | 9 +- .../flushcommon/syncmgr/sync_manager_test.go | 42 +-- internal/flushcommon/syncmgr/task_test.go | 2 +- .../util/checkpoint_updater.go | 2 +- .../util/checkpoint_updater_test.go | 3 +- .../util/rate_collector.go | 41 +- .../util/rate_collector_test.go | 4 +- .../{datanode => flushcommon}/util/tickler.go | 0 .../util/timetick_sender.go | 5 +- .../util/timetick_sender_test.go | 2 +- .../{datanode => flushcommon}/util/util.go | 26 +- .../writebuffer/bf_write_buffer_test.go | 2 +- .../writebuffer/l0_write_buffer.go | 2 +- internal/mocks/mock_datacoord.go | 255 ++++++++----- internal/mocks/mock_datacoord_client.go | 352 +++++++++++------- .../server/mock_flusher/mock_Flusher.go | 242 ++++++++++++ internal/proto/data_coord.proto | 12 + .../streamingnode/server/flusher/flusher.go | 40 ++ .../flusher/flusherimpl/flusher_impl.go | 206 ++++++++++ .../flusher/flusherimpl/flusher_impl_test.go | 185 +++++++++ .../flusherimpl/flushmsg_handler_impl.go | 36 ++ .../flusherimpl/flushmsg_handler_impl_test.go | 42 +++ .../flusher/flusherimpl/pipeline_params.go | 60 +++ .../server/flusher/flushmsg_handler.go | 21 ++ .../server/resource/idalloc/mallocator.go | 61 +++ .../streamingnode/server/resource/resource.go | 58 ++- .../server/wal/adaptor/builder.go | 2 + .../server/wal/adaptor/message_handler.go | 6 +- .../server/wal/interceptors/ddl/builder.go | 39 ++ .../wal/interceptors/ddl/ddl_interceptor.go | 48 +++ .../server/walmanager/manager_impl_test.go | 15 + .../server/walmanager/wal_lifetime.go | 18 +- .../server/walmanager/wal_lifetime_test.go | 17 + .../util/message/adaptor/message_id.go | 20 + pkg/util/paramtable/component_param.go | 10 +- 83 files changed, 2031 insertions(+), 591 deletions(-) rename internal/{datanode => flushcommon}/broker/broker.go (100%) rename internal/{datanode => flushcommon}/broker/datacoord.go (100%) rename internal/{datanode => flushcommon}/broker/datacoord_test.go (100%) rename internal/{datanode => flushcommon}/broker/mock_broker.go (100%) rename internal/{datanode => flushcommon}/io/binlog_io.go (100%) rename internal/{datanode => flushcommon}/io/binlog_io_test.go (100%) rename internal/{datanode => flushcommon}/io/io_pool.go (100%) rename internal/{datanode => flushcommon}/io/io_pool_test.go (100%) rename internal/{datanode => flushcommon}/io/mock_binlogio.go (100%) rename internal/{datanode/util/testutils.go => flushcommon/pipeline/testutils_test.go} (94%) rename internal/{datanode => flushcommon}/util/checkpoint_updater.go (99%) rename internal/{datanode => flushcommon}/util/checkpoint_updater_test.go (96%) rename internal/{datanode => flushcommon}/util/rate_collector.go (71%) rename internal/{datanode => flushcommon}/util/rate_collector_test.go (93%) rename internal/{datanode => flushcommon}/util/tickler.go (100%) rename internal/{datanode => flushcommon}/util/timetick_sender.go (96%) rename internal/{datanode => flushcommon}/util/timetick_sender_test.go (98%) rename internal/{datanode => flushcommon}/util/util.go (84%) create mode 100644 internal/mocks/streamingnode/server/mock_flusher/mock_Flusher.go create mode 100644 internal/streamingnode/server/flusher/flusher.go create mode 100644 internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go create mode 100644 internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go create mode 100644 internal/streamingnode/server/flusher/flusherimpl/flushmsg_handler_impl.go create mode 100644 internal/streamingnode/server/flusher/flusherimpl/flushmsg_handler_impl_test.go create mode 100644 internal/streamingnode/server/flusher/flusherimpl/pipeline_params.go create mode 100644 internal/streamingnode/server/flusher/flushmsg_handler.go create mode 100644 internal/streamingnode/server/resource/idalloc/mallocator.go create mode 100644 internal/streamingnode/server/wal/interceptors/ddl/builder.go create mode 100644 internal/streamingnode/server/wal/interceptors/ddl/ddl_interceptor.go diff --git a/Makefile b/Makefile index f3daaff0b6..0e3ee5e947 100644 --- a/Makefile +++ b/Makefile @@ -507,19 +507,21 @@ generate-mockery-datacoord: getdeps generate-mockery-datanode: getdeps $(INSTALL_PATH)/mockery --name=Allocator --dir=$(PWD)/internal/datanode/allocator --output=$(PWD)/internal/datanode/allocator --filename=mock_allocator.go --with-expecter --structname=MockAllocator --outpkg=allocator --inpackage - $(INSTALL_PATH)/mockery --name=Broker --dir=$(PWD)/internal/datanode/broker --output=$(PWD)/internal/datanode/broker/ --filename=mock_broker.go --with-expecter --structname=MockBroker --outpkg=broker --inpackage - $(INSTALL_PATH)/mockery --name=MetaCache --dir=$(PWD)/internal/datanode/metacache --output=$(PWD)/internal/datanode/metacache --filename=mock_meta_cache.go --with-expecter --structname=MockMetaCache --outpkg=metacache --inpackage - $(INSTALL_PATH)/mockery --name=SyncManager --dir=$(PWD)/internal/datanode/syncmgr --output=$(PWD)/internal/datanode/syncmgr --filename=mock_sync_manager.go --with-expecter --structname=MockSyncManager --outpkg=syncmgr --inpackage - $(INSTALL_PATH)/mockery --name=MetaWriter --dir=$(PWD)/internal/datanode/syncmgr --output=$(PWD)/internal/datanode/syncmgr --filename=mock_meta_writer.go --with-expecter --structname=MockMetaWriter --outpkg=syncmgr --inpackage - $(INSTALL_PATH)/mockery --name=Serializer --dir=$(PWD)/internal/datanode/syncmgr --output=$(PWD)/internal/datanode/syncmgr --filename=mock_serializer.go --with-expecter --structname=MockSerializer --outpkg=syncmgr --inpackage - $(INSTALL_PATH)/mockery --name=Task --dir=$(PWD)/internal/datanode/syncmgr --output=$(PWD)/internal/datanode/syncmgr --filename=mock_task.go --with-expecter --structname=MockTask --outpkg=syncmgr --inpackage - $(INSTALL_PATH)/mockery --name=WriteBuffer --dir=$(PWD)/internal/datanode/writebuffer --output=$(PWD)/internal/datanode/writebuffer --filename=mock_write_buffer.go --with-expecter --structname=MockWriteBuffer --outpkg=writebuffer --inpackage - $(INSTALL_PATH)/mockery --name=BufferManager --dir=$(PWD)/internal/datanode/writebuffer --output=$(PWD)/internal/datanode/writebuffer --filename=mock_mananger.go --with-expecter --structname=MockBufferManager --outpkg=writebuffer --inpackage - $(INSTALL_PATH)/mockery --name=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/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-flushcommon: getdeps + $(INSTALL_PATH)/mockery --name=Broker --dir=$(PWD)/internal/flushcommon/broker --output=$(PWD)/internal/flushcommon/broker/ --filename=mock_broker.go --with-expecter --structname=MockBroker --outpkg=broker --inpackage + $(INSTALL_PATH)/mockery --name=MetaCache --dir=$(PWD)/internal/flushcommon/metacache --output=$(PWD)/internal/flushcommon/metacache --filename=mock_meta_cache.go --with-expecter --structname=MockMetaCache --outpkg=metacache --inpackage + $(INSTALL_PATH)/mockery --name=SyncManager --dir=$(PWD)/internal/flushcommon/syncmgr --output=$(PWD)/internal/flushcommon/syncmgr --filename=mock_sync_manager.go --with-expecter --structname=MockSyncManager --outpkg=syncmgr --inpackage + $(INSTALL_PATH)/mockery --name=MetaWriter --dir=$(PWD)/internal/flushcommon/syncmgr --output=$(PWD)/internal/flushcommon/syncmgr --filename=mock_meta_writer.go --with-expecter --structname=MockMetaWriter --outpkg=syncmgr --inpackage + $(INSTALL_PATH)/mockery --name=Serializer --dir=$(PWD)/internal/flushcommon/syncmgr --output=$(PWD)/internal/flushcommon/syncmgr --filename=mock_serializer.go --with-expecter --structname=MockSerializer --outpkg=syncmgr --inpackage + $(INSTALL_PATH)/mockery --name=Task --dir=$(PWD)/internal/flushcommon/syncmgr --output=$(PWD)/internal/flushcommon/syncmgr --filename=mock_task.go --with-expecter --structname=MockTask --outpkg=syncmgr --inpackage + $(INSTALL_PATH)/mockery --name=WriteBuffer --dir=$(PWD)/internal/flushcommon/writebuffer --output=$(PWD)/internal/flushcommon/writebuffer --filename=mock_write_buffer.go --with-expecter --structname=MockWriteBuffer --outpkg=writebuffer --inpackage + $(INSTALL_PATH)/mockery --name=BufferManager --dir=$(PWD)/internal/flushcommon/writebuffer --output=$(PWD)/internal/flushcommon/writebuffer --filename=mock_manager.go --with-expecter --structname=MockBufferManager --outpkg=writebuffer --inpackage + $(INSTALL_PATH)/mockery --name=BinlogIO --dir=$(PWD)/internal/flushcommon/io --output=$(PWD)/internal/flushcommon/io --filename=mock_binlogio.go --with-expecter --structname=MockBinlogIO --outpkg=io --inpackage + $(INSTALL_PATH)/mockery --name=FlowgraphManager --dir=$(PWD)/internal/flushcommon/pipeline --output=$(PWD)/internal/flushcommon/pipeline --filename=mock_fgmanager.go --with-expecter --structname=MockFlowgraphManager --outpkg=pipeline --inpackage + generate-mockery-metastore: getdeps $(INSTALL_PATH)/mockery --name=RootCoordCatalog --dir=$(PWD)/internal/metastore --output=$(PWD)/internal/metastore/mocks --filename=mock_rootcoord_catalog.go --with-expecter --structname=RootCoordCatalog --outpkg=mocks $(INSTALL_PATH)/mockery --name=DataCoordCatalog --dir=$(PWD)/internal/metastore --output=$(PWD)/internal/metastore/mocks --filename=mock_datacoord_catalog.go --with-expecter --structname=DataCoordCatalog --outpkg=mocks diff --git a/internal/.mockery.yaml b/internal/.mockery.yaml index c68667da37..d50d113192 100644 --- a/internal/.mockery.yaml +++ b/internal/.mockery.yaml @@ -20,6 +20,9 @@ packages: github.com/milvus-io/milvus/internal/streamingnode/client/handler/consumer: interfaces: Consumer: + github.com/milvus-io/milvus/internal/streamingnode/server/flusher: + interfaces: + Flusher: github.com/milvus-io/milvus/internal/streamingnode/server/wal: interfaces: OpenerBuilder: diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index b5f15505d7..7e23d91810 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -907,6 +907,42 @@ func (s *Server) GetRecoveryInfoV2(ctx context.Context, req *datapb.GetRecoveryI return resp, nil } +// GetChannelRecoveryInfo get recovery channel info. +// Called by: StreamingNode. +func (s *Server) GetChannelRecoveryInfo(ctx context.Context, req *datapb.GetChannelRecoveryInfoRequest) (*datapb.GetChannelRecoveryInfoResponse, error) { + log := log.Ctx(ctx).With( + zap.String("vchannel", req.GetVchannel()), + ) + log.Info("get channel recovery info request received") + resp := &datapb.GetChannelRecoveryInfoResponse{ + Status: merr.Success(), + } + if err := merr.CheckHealthy(s.GetStateCode()); err != nil { + resp.Status = merr.Status(err) + return resp, nil + } + collectionID := funcutil.GetCollectionIDFromVChannel(req.GetVchannel()) + collection, err := s.handler.GetCollection(ctx, collectionID) + if err != nil { + resp.Status = merr.Status(err) + return resp, nil + } + channel := NewRWChannel(req.GetVchannel(), collectionID, nil, collection.Schema, 0) // TODO: remove RWChannel, just use vchannel + collectionID + channelInfo := s.handler.GetDataVChanPositions(channel, allPartitionID) + log.Info("datacoord get channel recovery info", + zap.String("channel", channelInfo.GetChannelName()), + zap.Int("# of unflushed segments", len(channelInfo.GetUnflushedSegmentIds())), + zap.Int("# of flushed segments", len(channelInfo.GetFlushedSegmentIds())), + zap.Int("# of dropped segments", len(channelInfo.GetDroppedSegmentIds())), + zap.Int("# of indexed segments", len(channelInfo.GetIndexedSegmentIds())), + zap.Int("# of l0 segments", len(channelInfo.GetLevelZeroSegmentIds())), + ) + + resp.Info = channelInfo + resp.Schema = collection.Schema + return resp, nil +} + // GetFlushedSegments returns all segment matches provided criterion and in state Flushed or Dropped (compacted but not GCed yet) // If requested partition id < 0, ignores the partition id filter func (s *Server) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) { diff --git a/internal/datacoord/services_test.go b/internal/datacoord/services_test.go index 251fcd5544..3aa64919b9 100644 --- a/internal/datacoord/services_test.go +++ b/internal/datacoord/services_test.go @@ -1460,6 +1460,56 @@ func TestImportV2(t *testing.T) { }) } +func TestGetChannelRecoveryInfo(t *testing.T) { + ctx := context.Background() + + // server not healthy + s := &Server{} + s.stateCode.Store(commonpb.StateCode_Initializing) + resp, err := s.GetChannelRecoveryInfo(ctx, nil) + assert.NoError(t, err) + assert.NotEqual(t, int32(0), resp.GetStatus().GetCode()) + s.stateCode.Store(commonpb.StateCode_Healthy) + + // get collection failed + handler := NewNMockHandler(t) + handler.EXPECT().GetCollection(mock.Anything, mock.Anything). + Return(nil, errors.New("mock err")) + s.handler = handler + assert.NoError(t, err) + resp, err = s.GetChannelRecoveryInfo(ctx, &datapb.GetChannelRecoveryInfoRequest{ + Vchannel: "ch-1", + }) + assert.NoError(t, err) + assert.Error(t, merr.Error(resp.GetStatus())) + + // normal case + channelInfo := &datapb.VchannelInfo{ + CollectionID: 0, + ChannelName: "ch-1", + SeekPosition: nil, + UnflushedSegmentIds: []int64{1}, + FlushedSegmentIds: []int64{2}, + DroppedSegmentIds: []int64{3}, + IndexedSegmentIds: []int64{4}, + } + + handler = NewNMockHandler(t) + handler.EXPECT().GetCollection(mock.Anything, mock.Anything). + Return(&collectionInfo{Schema: &schemapb.CollectionSchema{}}, nil) + handler.EXPECT().GetDataVChanPositions(mock.Anything, mock.Anything).Return(channelInfo) + s.handler = handler + + assert.NoError(t, err) + resp, err = s.GetChannelRecoveryInfo(ctx, &datapb.GetChannelRecoveryInfoRequest{ + Vchannel: "ch-1", + }) + assert.NoError(t, err) + assert.Equal(t, int32(0), resp.GetStatus().GetCode()) + assert.NotNil(t, resp.GetSchema()) + assert.Equal(t, channelInfo, resp.GetInfo()) +} + type GcControlServiceSuite struct { suite.Suite diff --git a/internal/datanode/channel/channel_manager.go b/internal/datanode/channel/channel_manager.go index 0580e4e9e3..645063ae64 100644 --- a/internal/datanode/channel/channel_manager.go +++ b/internal/datanode/channel/channel_manager.go @@ -24,8 +24,8 @@ import ( "github.com/cockroachdb/errors" "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/flushcommon/pipeline" + "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/lifetime" @@ -241,7 +241,7 @@ type opRunner struct { watchFunc watchFunc guard sync.RWMutex - allOps map[util.UniqueID]*opInfo // opID -> tickler + allOps map[typeutil.UniqueID]*opInfo // opID -> tickler opsInQueue chan *datapb.ChannelWatchInfo resultCh chan *opState @@ -256,7 +256,7 @@ func NewOpRunner(channel string, pipelineParams *util.PipelineParams, releaseF r releaseFunc: releaseF, watchFunc: watchF, opsInQueue: make(chan *datapb.ChannelWatchInfo, 10), - allOps: make(map[util.UniqueID]*opInfo), + allOps: make(map[typeutil.UniqueID]*opInfo), resultCh: resultCh, closeCh: lifetime.NewSafeChan(), } @@ -277,13 +277,13 @@ func (r *opRunner) Start() { }() } -func (r *opRunner) FinishOp(opID util.UniqueID) { +func (r *opRunner) FinishOp(opID typeutil.UniqueID) { r.guard.Lock() defer r.guard.Unlock() delete(r.allOps, opID) } -func (r *opRunner) Exist(opID util.UniqueID) (progress int32, exists bool) { +func (r *opRunner) Exist(opID typeutil.UniqueID) (progress int32, exists bool) { r.guard.RLock() defer r.guard.RUnlock() info, ok := r.allOps[opID] @@ -423,7 +423,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState { } // releaseWithTimer will return ReleaseFailure after WatchTimeoutInterval -func (r *opRunner) releaseWithTimer(releaseFunc releaseFunc, channel string, opID util.UniqueID) *opState { +func (r *opRunner) releaseWithTimer(releaseFunc releaseFunc, channel string, opID typeutil.UniqueID) *opState { opState := &opState{ channel: channel, opID: opID, diff --git a/internal/datanode/channel/channel_manager_test.go b/internal/datanode/channel/channel_manager_test.go index a79fec7e12..3aa166cb40 100644 --- a/internal/datanode/channel/channel_manager_test.go +++ b/internal/datanode/channel/channel_manager_test.go @@ -25,15 +25,18 @@ import ( "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datanode/allocator" - "github.com/milvus-io/milvus/internal/datanode/broker" - "github.com/milvus-io/milvus/internal/datanode/util" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/pipeline" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" + util2 "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" "github.com/milvus-io/milvus/internal/proto/datapb" "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" @@ -43,10 +46,6 @@ import ( 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) } @@ -74,10 +73,10 @@ func (s *OpRunnerSuite) SetupTest() { Return(make(chan *msgstream.MsgPack), nil).Maybe() dispClient.EXPECT().Deregister(mock.Anything).Maybe() - s.pipelineParams = &util.PipelineParams{ + s.pipelineParams = &util2.PipelineParams{ Ctx: context.TODO(), Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 0}}, - CheckpointUpdater: util.NewChannelCheckpointUpdater(mockedBroker), + CheckpointUpdater: util2.NewChannelCheckpointUpdater(mockedBroker), WriteBufferManager: wbManager, Broker: mockedBroker, DispClient: dispClient, @@ -91,7 +90,7 @@ func (s *OpRunnerSuite) TestWatchWithTimer() { channel string = "ch-1" commuCh = make(chan *opState) ) - info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) + info := GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) mockReleaseFunc := func(channel string) { log.Info("mock release func") } @@ -111,13 +110,13 @@ func (s *OpRunnerSuite) TestWatchTimeout() { channel := "by-dev-rootcoord-dml-1000" paramtable.Get().Save(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key, "0.000001") defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key) - info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) + info := 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, param *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler) (*pipeline.DataSyncService, error) { + mockWatchFunc := func(ctx context.Context, param *util2.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util2.Tickler) (*pipeline.DataSyncService, error) { <-ctx.Done() sig <- struct{}{} return nil, errors.New("timeout") @@ -138,13 +137,13 @@ func (s *OpRunnerSuite) TestWatchTimeout() { type OpRunnerSuite struct { suite.Suite - pipelineParams *util.PipelineParams + pipelineParams *util2.PipelineParams } type ChannelManagerSuite struct { suite.Suite - pipelineParams *util.PipelineParams + pipelineParams *util2.PipelineParams manager *ChannelManagerImpl } @@ -160,7 +159,7 @@ func (s *ChannelManagerSuite) SetupTest() { mockedBroker := &broker.MockBroker{} mockedBroker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).Return([]*datapb.SegmentInfo{}, nil).Maybe() - s.pipelineParams = &util.PipelineParams{ + s.pipelineParams = &util2.PipelineParams{ Ctx: context.TODO(), Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 0}}, WriteBufferManager: wbManager, @@ -189,7 +188,7 @@ func (s *ChannelManagerSuite) TestReleaseStuck() { stuckSig <- struct{}{} } - info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) + info := GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) s.Require().Equal(0, s.manager.opRunners.Len()) err := s.manager.Submit(info) s.Require().NoError(err) @@ -199,7 +198,7 @@ func (s *ChannelManagerSuite) TestReleaseStuck() { s.manager.handleOpState(opState) - releaseInfo := util.GetWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease) + releaseInfo := GetWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease) paramtable.Get().Save(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key, "0.1") defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key) @@ -225,7 +224,7 @@ func (s *ChannelManagerSuite) TestReleaseStuck() { func (s *ChannelManagerSuite) TestSubmitIdempotent() { channel := "by-dev-rootcoord-dml-1" - info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) + info := GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) s.Require().Equal(0, s.manager.opRunners.Len()) for i := 0; i < 10; i++ { @@ -244,7 +243,7 @@ func (s *ChannelManagerSuite) TestSubmitIdempotent() { func (s *ChannelManagerSuite) TestSubmitSkip() { channel := "by-dev-rootcoord-dml-1" - info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) + info := GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) s.Require().Equal(0, s.manager.opRunners.Len()) err := s.manager.Submit(info) @@ -271,7 +270,7 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() { channel := "by-dev-rootcoord-dml-0" // watch - info := util.GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) + info := GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) err := s.manager.Submit(info) s.NoError(err) @@ -296,7 +295,7 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() { s.Equal(datapb.ChannelWatchState_WatchSuccess, resp.GetState()) // release - info = util.GetWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease) + info = GetWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease) err = s.manager.Submit(info) s.NoError(err) @@ -320,3 +319,34 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() { s.False(ok) s.Nil(runner) } + +func GetWatchInfoByOpID(opID typeutil.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/compaction/clustering_compactor.go b/internal/datanode/compaction/clustering_compactor.go index bb2ab6cb26..dda212cc6a 100644 --- a/internal/datanode/compaction/clustering_compactor.go +++ b/internal/datanode/compaction/clustering_compactor.go @@ -39,7 +39,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/allocator" - "github.com/milvus-io/milvus/internal/datanode/io" + "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/proto/clusteringpb" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" diff --git a/internal/datanode/compaction/clustering_compactor_test.go b/internal/datanode/compaction/clustering_compactor_test.go index 1b5aba34d1..05f79f5a8d 100644 --- a/internal/datanode/compaction/clustering_compactor_test.go +++ b/internal/datanode/compaction/clustering_compactor_test.go @@ -30,7 +30,7 @@ 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/allocator" - "github.com/milvus-io/milvus/internal/datanode/io" + "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" diff --git a/internal/datanode/compaction/compactor_common.go b/internal/datanode/compaction/compactor_common.go index b2cb1d7db2..0fe7505425 100644 --- a/internal/datanode/compaction/compactor_common.go +++ b/internal/datanode/compaction/compactor_common.go @@ -25,8 +25,8 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus/internal/allocator" - "github.com/milvus-io/milvus/internal/datanode/io" iter "github.com/milvus-io/milvus/internal/datanode/iterators" + "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" diff --git a/internal/datanode/compaction/l0_compactor.go b/internal/datanode/compaction/l0_compactor.go index f5ae0e5ce7..9eb7194fc9 100644 --- a/internal/datanode/compaction/l0_compactor.go +++ b/internal/datanode/compaction/l0_compactor.go @@ -28,7 +28,7 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus/internal/allocator" - "github.com/milvus-io/milvus/internal/datanode/io" + "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" "github.com/milvus-io/milvus/internal/proto/datapb" diff --git a/internal/datanode/compaction/l0_compactor_test.go b/internal/datanode/compaction/l0_compactor_test.go index 961cc6639e..98fa5cf234 100644 --- a/internal/datanode/compaction/l0_compactor_test.go +++ b/internal/datanode/compaction/l0_compactor_test.go @@ -28,7 +28,7 @@ import ( "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/io" + "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/proto/datapb" diff --git a/internal/datanode/compaction/mix_compactor.go b/internal/datanode/compaction/mix_compactor.go index 657b0bb55a..196cb3da8a 100644 --- a/internal/datanode/compaction/mix_compactor.go +++ b/internal/datanode/compaction/mix_compactor.go @@ -29,7 +29,7 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus/internal/allocator" - "github.com/milvus-io/milvus/internal/datanode/io" + "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" diff --git a/internal/datanode/compaction/mix_compactor_test.go b/internal/datanode/compaction/mix_compactor_test.go index f552d7b738..68cccfe330 100644 --- a/internal/datanode/compaction/mix_compactor_test.go +++ b/internal/datanode/compaction/mix_compactor_test.go @@ -30,7 +30,7 @@ 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/allocator" - "github.com/milvus-io/milvus/internal/datanode/io" + "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/etcdpb" diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index c6b634470d..871d138438 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -35,13 +35,14 @@ 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/util" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/pipeline" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" + util2 "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "github.com/milvus-io/milvus/internal/storage" @@ -99,8 +100,8 @@ type DataNode struct { segmentCache *util.Cache compactionExecutor compaction.Executor - timeTickSender *util.TimeTickSender - channelCheckpointUpdater *util.ChannelCheckpointUpdater + timeTickSender *util2.TimeTickSender + channelCheckpointUpdater *util2.ChannelCheckpointUpdater etcdCli *clientv3.Client address string @@ -233,14 +234,6 @@ func (node *DataNode) Init() error { node.broker = broker.NewCoordBroker(node.dataCoord, serverID) - err := util.InitGlobalRateCollector() - if err != nil { - log.Error("DataNode server init rateCollector failed", zap.Error(err)) - initError = err - return - } - log.Info("DataNode server init rateCollector done") - node.dispClient = msgdispatcher.NewClient(node.factory, typeutil.DataNodeRole, serverID) log.Info("DataNode server init dispatcher client done") @@ -263,19 +256,14 @@ func (node *DataNode) Init() error { } node.chunkManager = chunkManager - syncMgr, err := syncmgr.NewSyncManager(node.chunkManager) - if err != nil { - initError = err - log.Error("failed to create sync manager", zap.Error(err)) - return - } + syncMgr := syncmgr.NewSyncManager(node.chunkManager) node.syncMgr = syncMgr node.writeBufferManager = writebuffer.NewManager(syncMgr) node.importTaskMgr = importv2.NewTaskManager() node.importScheduler = importv2.NewScheduler(node.importTaskMgr) - node.channelCheckpointUpdater = util.NewChannelCheckpointUpdater(node.broker) + node.channelCheckpointUpdater = util2.NewChannelCheckpointUpdater(node.broker) node.flowgraphManager = pipeline.NewFlowgraphManager() log.Info("init datanode done", zap.String("Address", node.address)) @@ -326,7 +314,7 @@ func (node *DataNode) Start() error { go node.importScheduler.Start() - node.timeTickSender = util.NewTimeTickSender(node.broker, node.session.ServerID, + node.timeTickSender = util2.NewTimeTickSender(node.broker, node.session.ServerID, retry.Attempts(20), retry.Sleep(time.Millisecond*100)) node.timeTickSender.Start() @@ -420,8 +408,8 @@ func (node *DataNode) GetSession() *sessionutil.Session { return node.session } -func getPipelineParams(node *DataNode) *util.PipelineParams { - return &util.PipelineParams{ +func getPipelineParams(node *DataNode) *util2.PipelineParams { + return &util2.PipelineParams{ Ctx: node.ctx, Broker: node.broker, SyncMgr: node.syncMgr, diff --git a/internal/datanode/data_node_test.go b/internal/datanode/data_node_test.go index 9cc78c2309..d642f90c44 100644 --- a/internal/datanode/data_node_test.go +++ b/internal/datanode/data_node_test.go @@ -31,11 +31,12 @@ 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/util" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/pipeline" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" + util2 "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" + "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/types" @@ -72,11 +73,6 @@ 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())) - err = util.InitGlobalRateCollector() - if err != nil { - panic("init test failed, err = " + err.Error()) - } - code := t.Run() os.Exit(code) } @@ -92,10 +88,9 @@ func NewIDLEDataNodeMock(ctx context.Context, pkType schemapb.DataType) *DataNod 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.timeTickSender = util2.NewTimeTickSender(broker, 0) + syncMgr := syncmgr.NewSyncManager(node.chunkManager) node.syncMgr = syncMgr node.writeBufferManager = writebuffer.NewManager(syncMgr) @@ -145,7 +140,7 @@ func TestDataNode(t *testing.T) { description string }{ {nil, false, "nil input"}, - {&util.RootCoordFactory{}, true, "valid input"}, + {mocks.NewMockRootCoordClient(t), true, "valid input"}, } for _, test := range tests { @@ -168,7 +163,7 @@ func TestDataNode(t *testing.T) { description string }{ {nil, false, "nil input"}, - {&util.DataCoordFactory{}, true, "valid input"}, + {mocks.NewMockDataCoordClient(t), true, "valid input"}, } for _, test := range tests { @@ -205,10 +200,10 @@ func TestDataNode(t *testing.T) { req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.SystemInfoMetrics) assert.NoError(t, err) - util.DeregisterRateCollector(metricsinfo.InsertConsumeThroughput) + util2.DeregisterRateCollector(metricsinfo.InsertConsumeThroughput) resp, err := emptyNode.getSystemInfoMetrics(context.TODO(), req) assert.NoError(t, err) assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) - util.RegisterRateCollector(metricsinfo.InsertConsumeThroughput) + util2.RegisterRateCollector(metricsinfo.InsertConsumeThroughput) }) } diff --git a/internal/datanode/metrics_info.go b/internal/datanode/metrics_info.go index f7a6bc2219..8ae42196c2 100644 --- a/internal/datanode/metrics_info.go +++ b/internal/datanode/metrics_info.go @@ -20,7 +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/internal/flushcommon/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" @@ -34,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 := util.RateCol.Rate(label, ratelimitutil.DefaultAvgDuration) + rate, err2 := util.GetRateCollector().Rate(label, ratelimitutil.DefaultAvgDuration) if err2 != nil { err = err2 return @@ -50,7 +50,7 @@ func (node *DataNode) getQuotaMetrics() (*metricsinfo.DataNodeQuotaMetrics, erro return nil, err } - minFGChannel, minFGTt := util.RateCol.GetMinFlowGraphTt() + minFGChannel, minFGTt := util.GetRateCollector().GetMinFlowGraphTt() return &metricsinfo.DataNodeQuotaMetrics{ Hms: metricsinfo.HardwareMetrics{}, Rms: rms, diff --git a/internal/datanode/services.go b/internal/datanode/services.go index add9b9793e..874509233c 100644 --- a/internal/datanode/services.go +++ b/internal/datanode/services.go @@ -30,7 +30,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/datanode/importv2" - "github.com/milvus-io/milvus/internal/datanode/io" + "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" "github.com/milvus-io/milvus/internal/proto/datapb" diff --git a/internal/datanode/services_test.go b/internal/datanode/services_test.go index 3fd1490e25..2f71dc654f 100644 --- a/internal/datanode/services_test.go +++ b/internal/datanode/services_test.go @@ -33,11 +33,11 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" allocator2 "github.com/milvus-io/milvus/internal/allocator" "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/util" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/pipeline" + "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/storage" diff --git a/internal/datanode/util/cache.go b/internal/datanode/util/cache.go index 9da7031970..6c3933c3bf 100644 --- a/internal/datanode/util/cache.go +++ b/internal/datanode/util/cache.go @@ -28,33 +28,33 @@ import ( // After the flush procedure, whether the segment successfully flushed or not, // it'll be removed from the cache. So if flush failed, the secondary flush can be triggered. type Cache struct { - *typeutil.ConcurrentSet[UniqueID] + *typeutil.ConcurrentSet[typeutil.UniqueID] } // NewCache returns a new Cache func NewCache() *Cache { return &Cache{ - ConcurrentSet: typeutil.NewConcurrentSet[UniqueID](), + ConcurrentSet: typeutil.NewConcurrentSet[typeutil.UniqueID](), } } // checkIfCached returns whether unique id is in cache -func (c *Cache) checkIfCached(key UniqueID) bool { +func (c *Cache) checkIfCached(key typeutil.UniqueID) bool { return c.Contain(key) } // Cache caches a specific ID into the cache -func (c *Cache) Cache(ID UniqueID) { +func (c *Cache) Cache(ID typeutil.UniqueID) { c.Insert(ID) } // checkOrCache returns true if `key` is present. // Otherwise, it returns false and stores `key` into cache. -func (c *Cache) checkOrCache(key UniqueID) bool { +func (c *Cache) checkOrCache(key typeutil.UniqueID) bool { return !c.Insert(key) } // Remove removes a set of IDs from the cache -func (c *Cache) Remove(IDs ...UniqueID) { +func (c *Cache) Remove(IDs ...typeutil.UniqueID) { c.ConcurrentSet.Remove(IDs...) } diff --git a/internal/datanode/util/cache_test.go b/internal/datanode/util/cache_test.go index 3bcfdabbf4..5fced5177d 100644 --- a/internal/datanode/util/cache_test.go +++ b/internal/datanode/util/cache_test.go @@ -23,14 +23,11 @@ import ( "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus/pkg/util/paramtable" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) func TestMain(t *testing.M) { paramtable.Init() - err := InitGlobalRateCollector() - if err != nil { - panic("init test failed, err = " + err.Error()) - } code := t.Run() os.Exit(code) } @@ -40,13 +37,13 @@ func TestSegmentCache(t *testing.T) { assert.False(t, segCache.checkIfCached(0)) - segCache.Cache(UniqueID(0)) + segCache.Cache(typeutil.UniqueID(0)) assert.True(t, segCache.checkIfCached(0)) - assert.False(t, segCache.checkOrCache(UniqueID(1))) + assert.False(t, segCache.checkOrCache(typeutil.UniqueID(1))) assert.True(t, segCache.checkIfCached(1)) - assert.True(t, segCache.checkOrCache(UniqueID(1))) + assert.True(t, segCache.checkOrCache(typeutil.UniqueID(1))) - segCache.Remove(UniqueID(0)) + segCache.Remove(typeutil.UniqueID(0)) assert.False(t, segCache.checkIfCached(0)) } diff --git a/internal/distributed/datacoord/client/client.go b/internal/distributed/datacoord/client/client.go index b0a1039522..df5fecb1af 100644 --- a/internal/distributed/datacoord/client/client.go +++ b/internal/distributed/datacoord/client/client.go @@ -347,6 +347,18 @@ func (c *Client) GetRecoveryInfoV2(ctx context.Context, req *datapb.GetRecoveryI }) } +// GetChannelRecoveryInfo returns the corresponding vchannel info. +func (c *Client) GetChannelRecoveryInfo(ctx context.Context, req *datapb.GetChannelRecoveryInfoRequest, opts ...grpc.CallOption) (*datapb.GetChannelRecoveryInfoResponse, error) { + req = typeutil.Clone(req) + commonpbutil.UpdateMsgBase( + req.GetBase(), + commonpbutil.FillMsgBaseFromClient(paramtable.GetNodeID(), commonpbutil.WithTargetID(c.sess.ServerID)), + ) + return wrapGrpcCall(ctx, c, func(client datapb.DataCoordClient) (*datapb.GetChannelRecoveryInfoResponse, error) { + return client.GetChannelRecoveryInfo(ctx, req) + }) +} + // GetFlushedSegments returns flushed segment list of requested collection/parition // // ctx is the context to control request deadline and cancellation diff --git a/internal/distributed/datacoord/client/client_test.go b/internal/distributed/datacoord/client/client_test.go index 2cb56dbd44..c46dab7235 100644 --- a/internal/distributed/datacoord/client/client_test.go +++ b/internal/distributed/datacoord/client/client_test.go @@ -2271,3 +2271,50 @@ func Test_ListIndexes(t *testing.T) { _, err = client.ListIndexes(ctx, &indexpb.ListIndexesRequest{}) assert.ErrorIs(t, err, context.Canceled) } + +func Test_GetChannelRecoveryInfo(t *testing.T) { + paramtable.Init() + + ctx := context.Background() + client, err := NewClient(ctx) + assert.NoError(t, err) + assert.NotNil(t, client) + defer client.Close() + + mockDC := mocks.NewMockDataCoordClient(t) + mockGrpcClient := mocks.NewMockGrpcClient[datapb.DataCoordClient](t) + mockGrpcClient.EXPECT().Close().Return(nil) + mockGrpcClient.EXPECT().ReCall(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, f func(datapb.DataCoordClient) (interface{}, error)) (interface{}, error) { + return f(mockDC) + }) + client.(*Client).grpcClient = mockGrpcClient + + // test success + mockDC.EXPECT().GetChannelRecoveryInfo(mock.Anything, mock.Anything).Return(&datapb.GetChannelRecoveryInfoResponse{ + Status: merr.Success(), + }, nil).Once() + _, err = client.GetChannelRecoveryInfo(ctx, &datapb.GetChannelRecoveryInfoRequest{}) + assert.Nil(t, err) + + // test return error status + mockDC.EXPECT().GetChannelRecoveryInfo(mock.Anything, mock.Anything).Return( + &datapb.GetChannelRecoveryInfoResponse{ + Status: merr.Status(merr.ErrServiceNotReady), + }, nil).Once() + + rsp, err := client.GetChannelRecoveryInfo(ctx, &datapb.GetChannelRecoveryInfoRequest{}) + + assert.Nil(t, err) + assert.False(t, merr.Ok(rsp.GetStatus())) + + // test return error + mockDC.EXPECT().GetChannelRecoveryInfo(mock.Anything, mock.Anything).Return(nil, mockErr).Once() + _, err = client.GetChannelRecoveryInfo(ctx, &datapb.GetChannelRecoveryInfoRequest{}) + assert.Error(t, err) + + // test ctx done + ctx, cancel := context.WithCancel(ctx) + cancel() + _, err = client.GetChannelRecoveryInfo(ctx, &datapb.GetChannelRecoveryInfoRequest{}) + assert.ErrorIs(t, err, context.Canceled) +} diff --git a/internal/distributed/datacoord/service.go b/internal/distributed/datacoord/service.go index baa19c62ab..e1c013e119 100644 --- a/internal/distributed/datacoord/service.go +++ b/internal/distributed/datacoord/service.go @@ -348,6 +348,11 @@ func (s *Server) GetRecoveryInfoV2(ctx context.Context, req *datapb.GetRecoveryI return s.dataCoord.GetRecoveryInfoV2(ctx, req) } +// GetChannelRecoveryInfo gets the corresponding vchannel info. +func (s *Server) GetChannelRecoveryInfo(ctx context.Context, req *datapb.GetChannelRecoveryInfoRequest) (*datapb.GetChannelRecoveryInfoResponse, error) { + return s.dataCoord.GetChannelRecoveryInfo(ctx, req) +} + // GetFlushedSegments get all flushed segments of a partition func (s *Server) GetFlushedSegments(ctx context.Context, req *datapb.GetFlushedSegmentsRequest) (*datapb.GetFlushedSegmentsResponse, error) { return s.dataCoord.GetFlushedSegments(ctx, req) diff --git a/internal/distributed/datacoord/service_test.go b/internal/distributed/datacoord/service_test.go index 9a600a4687..955b7476ea 100644 --- a/internal/distributed/datacoord/service_test.go +++ b/internal/distributed/datacoord/service_test.go @@ -136,6 +136,13 @@ func Test_NewServer(t *testing.T) { assert.NotNil(t, resp) }) + t.Run("GetChannelRecoveryInfo", func(t *testing.T) { + mockDataCoord.EXPECT().GetChannelRecoveryInfo(mock.Anything, mock.Anything).Return(&datapb.GetChannelRecoveryInfoResponse{}, nil) + resp, err := server.GetChannelRecoveryInfo(ctx, nil) + assert.NoError(t, err) + assert.NotNil(t, resp) + }) + t.Run("GetFlushedSegments", func(t *testing.T) { mockDataCoord.EXPECT().GetFlushedSegments(mock.Anything, mock.Anything).Return(&datapb.GetFlushedSegmentsResponse{}, nil) resp, err := server.GetFlushedSegments(ctx, nil) diff --git a/internal/datanode/broker/broker.go b/internal/flushcommon/broker/broker.go similarity index 100% rename from internal/datanode/broker/broker.go rename to internal/flushcommon/broker/broker.go diff --git a/internal/datanode/broker/datacoord.go b/internal/flushcommon/broker/datacoord.go similarity index 100% rename from internal/datanode/broker/datacoord.go rename to internal/flushcommon/broker/datacoord.go diff --git a/internal/datanode/broker/datacoord_test.go b/internal/flushcommon/broker/datacoord_test.go similarity index 100% rename from internal/datanode/broker/datacoord_test.go rename to internal/flushcommon/broker/datacoord_test.go diff --git a/internal/datanode/broker/mock_broker.go b/internal/flushcommon/broker/mock_broker.go similarity index 100% rename from internal/datanode/broker/mock_broker.go rename to internal/flushcommon/broker/mock_broker.go diff --git a/internal/datanode/io/binlog_io.go b/internal/flushcommon/io/binlog_io.go similarity index 100% rename from internal/datanode/io/binlog_io.go rename to internal/flushcommon/io/binlog_io.go diff --git a/internal/datanode/io/binlog_io_test.go b/internal/flushcommon/io/binlog_io_test.go similarity index 100% rename from internal/datanode/io/binlog_io_test.go rename to internal/flushcommon/io/binlog_io_test.go diff --git a/internal/datanode/io/io_pool.go b/internal/flushcommon/io/io_pool.go similarity index 100% rename from internal/datanode/io/io_pool.go rename to internal/flushcommon/io/io_pool.go diff --git a/internal/datanode/io/io_pool_test.go b/internal/flushcommon/io/io_pool_test.go similarity index 100% rename from internal/datanode/io/io_pool_test.go rename to internal/flushcommon/io/io_pool_test.go diff --git a/internal/datanode/io/mock_binlogio.go b/internal/flushcommon/io/mock_binlogio.go similarity index 100% rename from internal/datanode/io/mock_binlogio.go rename to internal/flushcommon/io/mock_binlogio.go diff --git a/internal/flushcommon/pipeline/data_sync_service.go b/internal/flushcommon/pipeline/data_sync_service.go index ba60584445..0382295854 100644 --- a/internal/flushcommon/pipeline/data_sync_service.go +++ b/internal/flushcommon/pipeline/data_sync_service.go @@ -22,12 +22,12 @@ import ( "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/datanode/broker" "github.com/milvus-io/milvus/internal/datanode/compaction" - "github.com/milvus-io/milvus/internal/datanode/io" - "github.com/milvus-io/milvus/internal/datanode/util" + "github.com/milvus-io/milvus/internal/flushcommon/broker" + "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" + "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" @@ -48,12 +48,12 @@ type DataSyncService struct { cancelFn context.CancelFunc metacache metacache.MetaCache opID int64 - collectionID util.UniqueID // collection id of vchan for which this data sync service serves + collectionID typeutil.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 util.UniqueID + serverID typeutil.UniqueID fg *flowgraph.TimeTickedFlowGraph // internal flowgraph processes insert/delta messages @@ -71,10 +71,10 @@ type DataSyncService struct { type nodeConfig struct { msFactory msgstream.Factory // msgStream factory - collectionID util.UniqueID + collectionID typeutil.UniqueID vChannelName string metacache metacache.MetaCache - serverID util.UniqueID + serverID typeutil.UniqueID } // Start the flow graph in dataSyncService @@ -109,7 +109,9 @@ func (dsService *DataSyncService) close() { ) if dsService.fg != nil { log.Info("dataSyncService closing flowgraph") - dsService.dispClient.Deregister(dsService.vchannelName) + if dsService.dispClient != nil { + dsService.dispClient.Deregister(dsService.vchannelName) + } dsService.fg.Close() log.Info("dataSyncService flowgraph closed") } @@ -156,7 +158,9 @@ func initMetaCache(initCtx context.Context, chunkManager storage.ChunkManager, i return nil, err } segmentPks.Insert(segment.GetID(), stats) - tickler.Inc() + if tickler != nil { + tickler.Inc() + } return struct{}{}, nil }) @@ -185,18 +189,25 @@ func initMetaCache(initCtx context.Context, chunkManager storage.ChunkManager, i return metacache, nil } -func getServiceWithChannel(initCtx context.Context, params *util.PipelineParams, info *datapb.ChannelWatchInfo, metacache metacache.MetaCache, unflushed, flushed []*datapb.SegmentInfo) (*DataSyncService, error) { +func getServiceWithChannel(initCtx context.Context, params *util.PipelineParams, + info *datapb.ChannelWatchInfo, metacache metacache.MetaCache, + unflushed, flushed []*datapb.SegmentInfo, input <-chan *msgstream.MsgPack, +) (*DataSyncService, error) { var ( channelName = info.GetVchan().GetChannelName() collectionID = info.GetVchan().GetCollectionID() ) + serverID := paramtable.GetNodeID() + if params.Session != nil { + serverID = params.Session.ServerID + } config := &nodeConfig{ msFactory: params.MsgStreamFactory, collectionID: collectionID, vChannelName: channelName, metacache: metacache, - serverID: params.Session.ServerID, + serverID: serverID, } err := params.WriteBufferManager.Register(channelName, metacache, @@ -236,7 +247,7 @@ func getServiceWithChannel(initCtx context.Context, params *util.PipelineParams, // init flowgraph fg := flowgraph.NewTimeTickedFlowGraph(params.Ctx) - dmStreamNode, err := newDmInputNode(initCtx, params.DispClient, info.GetVchan().GetSeekPosition(), config) + dmStreamNode, err := newDmInputNode(initCtx, params.DispClient, info.GetVchan().GetSeekPosition(), config, input) if err != nil { return nil, err } @@ -249,6 +260,7 @@ func getServiceWithChannel(initCtx context.Context, params *util.PipelineParams, flushed, unflushed, params.CompactionExecutor, + params.FlushMsgHandler, ) if err != nil { return nil, err @@ -288,7 +300,36 @@ func NewDataSyncService(initCtx context.Context, pipelineParams *util.PipelinePa return nil, err } - return getServiceWithChannel(initCtx, pipelineParams, info, metaCache, unflushedSegmentInfos, flushedSegmentInfos) + return getServiceWithChannel(initCtx, pipelineParams, info, metaCache, unflushedSegmentInfos, flushedSegmentInfos, nil) +} + +func NewStreamingNodeDataSyncService(initCtx context.Context, pipelineParams *util.PipelineParams, info *datapb.ChannelWatchInfo, input <-chan *msgstream.MsgPack) (*DataSyncService, error) { + // recover segment checkpoints + var ( + err error + unflushedSegmentInfos []*datapb.SegmentInfo + flushedSegmentInfos []*datapb.SegmentInfo + ) + if len(info.GetVchan().GetUnflushedSegmentIds()) > 0 { + unflushedSegmentInfos, err = pipelineParams.Broker.GetSegmentInfo(initCtx, info.GetVchan().GetUnflushedSegmentIds()) + if err != nil { + return nil, err + } + } + if len(info.GetVchan().GetFlushedSegmentIds()) > 0 { + flushedSegmentInfos, err = pipelineParams.Broker.GetSegmentInfo(initCtx, info.GetVchan().GetFlushedSegmentIds()) + if err != nil { + return nil, err + } + } + + // init metaCache meta + metaCache, err := initMetaCache(initCtx, pipelineParams.ChunkManager, info, nil, unflushedSegmentInfos, flushedSegmentInfos) + if err != nil { + return nil, err + } + + return getServiceWithChannel(initCtx, pipelineParams, info, metaCache, unflushedSegmentInfos, flushedSegmentInfos, input) } func NewDataSyncServiceWithMetaCache(metaCache metacache.MetaCache) *DataSyncService { diff --git a/internal/flushcommon/pipeline/data_sync_service_test.go b/internal/flushcommon/pipeline/data_sync_service_test.go index 357fbbcf4a..1a55231b0e 100644 --- a/internal/flushcommon/pipeline/data_sync_service_test.go +++ b/internal/flushcommon/pipeline/data_sync_service_test.go @@ -32,10 +32,10 @@ import ( "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/allocator" - "github.com/milvus-io/milvus/internal/datanode/broker" - "github.com/milvus-io/milvus/internal/datanode/util" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" + util2 "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/proto/datapb" @@ -125,16 +125,16 @@ type testInfo struct { channelNil bool inMsgFactory dependency.Factory - collID util.UniqueID + collID typeutil.UniqueID chanName string - ufCollID util.UniqueID - ufSegID util.UniqueID + ufCollID typeutil.UniqueID + ufSegID typeutil.UniqueID ufchanName string ufNor int64 - fCollID util.UniqueID - fSegID util.UniqueID + fCollID typeutil.UniqueID + fSegID typeutil.UniqueID fchanName string fNor int64 @@ -202,11 +202,10 @@ func TestDataSyncService_newDataSyncService(t *testing.T) { }) }, nil) - pipelineParams := &util.PipelineParams{ + pipelineParams := &util2.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), @@ -218,7 +217,7 @@ func TestDataSyncService_newDataSyncService(t *testing.T) { ctx, pipelineParams, getWatchInfo(test), - util.NewTickler(), + util2.NewTickler(), ) if !test.isValidCase { @@ -238,14 +237,14 @@ func TestDataSyncService_newDataSyncService(t *testing.T) { func TestGetChannelWithTickler(t *testing.T) { channelName := "by-dev-rootcoord-dml-0" - info := util.GetWatchInfoByOpID(100, channelName, datapb.ChannelWatchState_ToWatch) + info := 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) + meta := NewMetaFactory().GetCollectionMeta(1, "test_collection", schemapb.DataType_Int64) info.Schema = meta.GetSchema() - pipelineParams := &util.PipelineParams{ + pipelineParams := &util2.PipelineParams{ Ctx: context.TODO(), Broker: broker.NewMockBroker(t), ChunkManager: chunkManager, @@ -289,7 +288,7 @@ func TestGetChannelWithTickler(t *testing.T) { }, } - metaCache, err := getMetaCacheWithTickler(context.TODO(), pipelineParams, info, util.NewTickler(), unflushed, flushed) + metaCache, err := getMetaCacheWithTickler(context.TODO(), pipelineParams, info, util2.NewTickler(), unflushed, flushed) assert.NoError(t, err) assert.NotNil(t, metaCache) assert.Equal(t, int64(1), metaCache.Collection()) @@ -299,14 +298,14 @@ func TestGetChannelWithTickler(t *testing.T) { type DataSyncServiceSuite struct { suite.Suite - util.MockDataSuiteBase + MockDataSuiteBase - pipelineParams *util.PipelineParams // node param + pipelineParams *util2.PipelineParams // node param chunkManager *mocks.ChunkManager broker *broker.MockBroker allocator *allocator.MockAllocator wbManager *writebuffer.MockBufferManager - channelCheckpointUpdater *util.ChannelCheckpointUpdater + channelCheckpointUpdater *util2.ChannelCheckpointUpdater factory *dependency.MockFactory ms *msgstream.MockMsgStream msChan chan *msgstream.MsgPack @@ -328,7 +327,7 @@ func (s *DataSyncServiceSuite) SetupTest() { paramtable.Get().Save(paramtable.Get().DataNodeCfg.ChannelCheckpointUpdateTickInSeconds.Key, "0.01") defer paramtable.Get().Save(paramtable.Get().DataNodeCfg.ChannelCheckpointUpdateTickInSeconds.Key, "10") - s.channelCheckpointUpdater = util.NewChannelCheckpointUpdater(s.broker) + s.channelCheckpointUpdater = util2.NewChannelCheckpointUpdater(s.broker) go s.channelCheckpointUpdater.Start() s.msChan = make(chan *msgstream.MsgPack, 1) @@ -340,17 +339,16 @@ func (s *DataSyncServiceSuite) SetupTest() { s.ms.EXPECT().Chan().Return(s.msChan) s.ms.EXPECT().Close().Return() - s.pipelineParams = &util.PipelineParams{ + s.pipelineParams = &util2.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), + TimeTickSender: util2.NewTimeTickSender(s.broker, 0), DispClient: msgdispatcher.NewClient(s.factory, typeutil.DataNodeRole, 1), } } @@ -359,8 +357,8 @@ func (s *DataSyncServiceSuite) TestStartStop() { var ( insertChannelName = fmt.Sprintf("by-dev-rootcoord-dml-%d", rand.Int()) - Factory = &util.MetaFactory{} - collMeta = Factory.GetCollectionMeta(util.UniqueID(0), "coll1", schemapb.DataType_Int64) + Factory = &MetaFactory{} + collMeta = Factory.GetCollectionMeta(typeutil.UniqueID(0), "coll1", schemapb.DataType_Int64) ) ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -429,7 +427,7 @@ func (s *DataSyncServiceSuite) TestStartStop() { ctx, s.pipelineParams, watchInfo, - util.NewTickler(), + util2.NewTickler(), ) s.Require().NoError(err) s.Require().NotNil(sync) @@ -437,13 +435,13 @@ func (s *DataSyncServiceSuite) TestStartStop() { sync.Start() defer sync.close() - timeRange := util.TimeRange{ + timeRange := util2.TimeRange{ TimestampMin: 0, TimestampMax: math.MaxUint64 - 1, } msgTs := tsoutil.GetCurrentTime() - dataFactory := util.NewDataFactory() + dataFactory := NewDataFactory() insertMessages := dataFactory.GetMsgStreamTsInsertMsgs(2, insertChannelName, msgTs) msgPack := msgstream.MsgPack{ @@ -472,7 +470,7 @@ func (s *DataSyncServiceSuite) TestStartStop() { TimeTickMsg: &msgpb.TimeTickMsg{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_TimeTick, - MsgID: util.UniqueID(0), + MsgID: typeutil.UniqueID(0), Timestamp: tsoutil.GetCurrentTime(), SourceID: 0, }, diff --git a/internal/flushcommon/pipeline/flow_graph_dd_node.go b/internal/flushcommon/pipeline/flow_graph_dd_node.go index b303680ab2..213c08aeb3 100644 --- a/internal/flushcommon/pipeline/flow_graph_dd_node.go +++ b/internal/flushcommon/pipeline/flow_graph_dd_node.go @@ -29,14 +29,16 @@ 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/flushcommon/util" "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/streamingnode/server/flusher" "github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/mq/msgstream" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) // make sure ddNode implements flowgraph.Node @@ -62,15 +64,16 @@ type ddNode struct { BaseNode ctx context.Context - collectionID util.UniqueID + collectionID typeutil.UniqueID vChannelName string dropMode atomic.Value compactionExecutor compaction.Executor + flushMsgHandler flusher.FlushMsgHandler // for recovery - growingSegInfo map[util.UniqueID]*datapb.SegmentInfo // segmentID - sealedSegInfo map[util.UniqueID]*datapb.SegmentInfo // segmentID + growingSegInfo map[typeutil.UniqueID]*datapb.SegmentInfo // segmentID + sealedSegInfo map[typeutil.UniqueID]*datapb.SegmentInfo // segmentID droppedSegmentIDs []int64 } @@ -181,7 +184,7 @@ func (ddn *ddNode) Operate(in []Msg) []Msg { continue } - util.RateCol.Add(metricsinfo.InsertConsumeThroughput, float64(proto.Size(imsg.InsertRequest))) + util.GetRateCollector().Add(metricsinfo.InsertConsumeThroughput, float64(proto.Size(imsg.InsertRequest))) metrics.DataNodeConsumeBytesCount. WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.InsertLabel). @@ -215,7 +218,7 @@ func (ddn *ddNode) Operate(in []Msg) []Msg { } log.Debug("DDNode receive delete messages", zap.String("channel", ddn.vChannelName), zap.Int64("numRows", dmsg.NumRows)) - util.RateCol.Add(metricsinfo.DeleteConsumeThroughput, float64(proto.Size(dmsg.DeleteRequest))) + util.GetRateCollector().Add(metricsinfo.DeleteConsumeThroughput, float64(proto.Size(dmsg.DeleteRequest))) metrics.DataNodeConsumeBytesCount. WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.DeleteLabel). @@ -229,6 +232,11 @@ func (ddn *ddNode) Operate(in []Msg) []Msg { WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.DeleteLabel). Add(float64(dmsg.GetNumRows())) fgMsg.DeleteMessages = append(fgMsg.DeleteMessages, dmsg) + + case commonpb.MsgType_Flush: + if ddn.flushMsgHandler != nil { + ddn.flushMsgHandler(ddn.vChannelName, nil) + } } } @@ -270,7 +278,7 @@ func (ddn *ddNode) tryToFilterSegmentInsertMessages(msg *msgstream.InsertMsg) bo return false } -func (ddn *ddNode) isDropped(segID util.UniqueID) bool { +func (ddn *ddNode) isDropped(segID typeutil.UniqueID) bool { for _, droppedSegmentID := range ddn.droppedSegmentIDs { if droppedSegmentID == segID { return true @@ -283,8 +291,8 @@ func (ddn *ddNode) Close() { log.Info("Flowgraph DD Node closing") } -func newDDNode(ctx context.Context, collID util.UniqueID, vChannelName string, droppedSegmentIDs []util.UniqueID, - sealedSegments []*datapb.SegmentInfo, growingSegments []*datapb.SegmentInfo, executor compaction.Executor, +func newDDNode(ctx context.Context, collID typeutil.UniqueID, vChannelName string, droppedSegmentIDs []typeutil.UniqueID, + sealedSegments []*datapb.SegmentInfo, growingSegments []*datapb.SegmentInfo, executor compaction.Executor, handler flusher.FlushMsgHandler, ) (*ddNode, error) { baseNode := BaseNode{} baseNode.SetMaxQueueLength(paramtable.Get().DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32()) @@ -294,11 +302,12 @@ func newDDNode(ctx context.Context, collID util.UniqueID, vChannelName string, d ctx: ctx, BaseNode: baseNode, collectionID: collID, - sealedSegInfo: make(map[util.UniqueID]*datapb.SegmentInfo, len(sealedSegments)), - growingSegInfo: make(map[util.UniqueID]*datapb.SegmentInfo, len(growingSegments)), + sealedSegInfo: make(map[typeutil.UniqueID]*datapb.SegmentInfo, len(sealedSegments)), + growingSegInfo: make(map[typeutil.UniqueID]*datapb.SegmentInfo, len(growingSegments)), droppedSegmentIDs: droppedSegmentIDs, vChannelName: vChannelName, compactionExecutor: executor, + flushMsgHandler: handler, } dd.dropMode.Store(false) diff --git a/internal/flushcommon/pipeline/flow_graph_dd_node_test.go b/internal/flushcommon/pipeline/flow_graph_dd_node_test.go index 8e234457b1..cda9b1c701 100644 --- a/internal/flushcommon/pipeline/flow_graph_dd_node_test.go +++ b/internal/flushcommon/pipeline/flow_graph_dd_node_test.go @@ -27,10 +27,10 @@ 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" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) const ( @@ -64,9 +64,9 @@ func TestFlowGraph_DDNode_newDDNode(t *testing.T) { } var ( - collectionID = util.UniqueID(1) + collectionID = typeutil.UniqueID(1) channelName = fmt.Sprintf("by-dev-rootcoord-dml-%s", t.Name()) - droppedSegIDs = []util.UniqueID{} + droppedSegIDs = []typeutil.UniqueID{} ) for _, test := range tests { @@ -79,6 +79,7 @@ func TestFlowGraph_DDNode_newDDNode(t *testing.T) { test.inSealedSegs, test.inGrowingSegs, compaction.NewExecutor(), + nil, ) require.NoError(t, err) require.NotNil(t, ddNode) @@ -120,9 +121,9 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) { } // valid inputs tests := []struct { - ddnCollID util.UniqueID + ddnCollID typeutil.UniqueID - msgCollID util.UniqueID + msgCollID typeutil.UniqueID expectedChlen int description string @@ -170,22 +171,22 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) { t.Run("Test DDNode Operate DropPartition Msg", func(t *testing.T) { // valid inputs tests := []struct { - ddnCollID util.UniqueID + ddnCollID typeutil.UniqueID - msgCollID util.UniqueID - msgPartID util.UniqueID - expectOutput []util.UniqueID + msgCollID typeutil.UniqueID + msgPartID typeutil.UniqueID + expectOutput []typeutil.UniqueID description string }{ { 1, 1, 101, - []util.UniqueID{101}, + []typeutil.UniqueID{101}, "DropCollectionMsg collID == ddNode collID", }, { 1, 2, 101, - []util.UniqueID{}, + []typeutil.UniqueID{}, "DropCollectionMsg collID != ddNode collID", }, } @@ -220,12 +221,12 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) { }) t.Run("Test DDNode Operate and filter insert msg", func(t *testing.T) { - var collectionID util.UniqueID = 1 + var collectionID typeutil.UniqueID = 1 // Prepare ddNode states ddn := ddNode{ ctx: context.Background(), collectionID: collectionID, - droppedSegmentIDs: []util.UniqueID{100}, + droppedSegmentIDs: []typeutil.UniqueID{100}, } tsMessages := []msgstream.TsMsg{getInsertMsg(100, 10000), getInsertMsg(200, 20000)} @@ -237,10 +238,10 @@ func TestFlowGraph_DDNode_Operate(t *testing.T) { t.Run("Test DDNode Operate Delete Msg", func(t *testing.T) { tests := []struct { - ddnCollID util.UniqueID - inMsgCollID util.UniqueID + ddnCollID typeutil.UniqueID + inMsgCollID typeutil.UniqueID - MsgEndTs util.Timestamp + MsgEndTs typeutil.Timestamp expectedRtLen int description string @@ -283,16 +284,16 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { tests := []struct { description string - droppedSegIDs []util.UniqueID - sealedSegInfo map[util.UniqueID]*datapb.SegmentInfo - growingSegInfo map[util.UniqueID]*datapb.SegmentInfo + droppedSegIDs []typeutil.UniqueID + sealedSegInfo map[typeutil.UniqueID]*datapb.SegmentInfo + growingSegInfo map[typeutil.UniqueID]*datapb.SegmentInfo inMsg *msgstream.InsertMsg expected bool }{ { "test dropped segments true", - []util.UniqueID{100}, + []typeutil.UniqueID{100}, nil, nil, getInsertMsg(100, 10000), @@ -300,7 +301,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { }, { "test dropped segments true 2", - []util.UniqueID{100, 101, 102}, + []typeutil.UniqueID{100, 101, 102}, nil, nil, getInsertMsg(102, 10000), @@ -308,8 +309,8 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { }, { "test sealed segments msgTs <= segmentTs true", - []util.UniqueID{}, - map[util.UniqueID]*datapb.SegmentInfo{ + []typeutil.UniqueID{}, + map[typeutil.UniqueID]*datapb.SegmentInfo{ 200: getSegmentInfo(200, 50000), 300: getSegmentInfo(300, 50000), }, @@ -319,8 +320,8 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { }, { "test sealed segments msgTs <= segmentTs true", - []util.UniqueID{}, - map[util.UniqueID]*datapb.SegmentInfo{ + []typeutil.UniqueID{}, + map[typeutil.UniqueID]*datapb.SegmentInfo{ 200: getSegmentInfo(200, 50000), 300: getSegmentInfo(300, 50000), }, @@ -330,8 +331,8 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { }, { "test sealed segments msgTs > segmentTs false", - []util.UniqueID{}, - map[util.UniqueID]*datapb.SegmentInfo{ + []typeutil.UniqueID{}, + map[typeutil.UniqueID]*datapb.SegmentInfo{ 200: getSegmentInfo(200, 50000), 300: getSegmentInfo(300, 50000), }, @@ -341,9 +342,9 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { }, { "test growing segments msgTs <= segmentTs true", - []util.UniqueID{}, + []typeutil.UniqueID{}, nil, - map[util.UniqueID]*datapb.SegmentInfo{ + map[typeutil.UniqueID]*datapb.SegmentInfo{ 200: getSegmentInfo(200, 50000), 300: getSegmentInfo(300, 50000), }, @@ -352,9 +353,9 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { }, { "test growing segments msgTs > segmentTs false", - []util.UniqueID{}, + []typeutil.UniqueID{}, nil, - map[util.UniqueID]*datapb.SegmentInfo{ + map[typeutil.UniqueID]*datapb.SegmentInfo{ 200: getSegmentInfo(200, 50000), 300: getSegmentInfo(300, 50000), }, @@ -363,12 +364,12 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { }, { "test not exist", - []util.UniqueID{}, - map[util.UniqueID]*datapb.SegmentInfo{ + []typeutil.UniqueID{}, + map[typeutil.UniqueID]*datapb.SegmentInfo{ 400: getSegmentInfo(500, 50000), 500: getSegmentInfo(400, 50000), }, - map[util.UniqueID]*datapb.SegmentInfo{ + map[typeutil.UniqueID]*datapb.SegmentInfo{ 200: getSegmentInfo(200, 50000), 300: getSegmentInfo(300, 50000), }, @@ -378,7 +379,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { // for pChannel reuse on same collection { "test insert msg with different channelName", - []util.UniqueID{100}, + []typeutil.UniqueID{100}, nil, nil, getInsertMsgWithChannel(100, 10000, anotherChannelName), @@ -406,10 +407,10 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { description string segRemained bool - segTs util.Timestamp - msgTs util.Timestamp + segTs typeutil.Timestamp + msgTs typeutil.Timestamp - sealedSegInfo map[util.UniqueID]*datapb.SegmentInfo + sealedSegInfo map[typeutil.UniqueID]*datapb.SegmentInfo inMsg *msgstream.InsertMsg msgFiltered bool }{ @@ -418,7 +419,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { true, 50000, 10000, - map[util.UniqueID]*datapb.SegmentInfo{ + map[typeutil.UniqueID]*datapb.SegmentInfo{ 100: getSegmentInfo(100, 50000), 101: getSegmentInfo(101, 50000), }, @@ -430,7 +431,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { true, 50000, 10000, - map[util.UniqueID]*datapb.SegmentInfo{ + map[typeutil.UniqueID]*datapb.SegmentInfo{ 100: getSegmentInfo(100, 50000), 101: getSegmentInfo(101, 50000), }, @@ -442,7 +443,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { false, 50000, 10000, - map[util.UniqueID]*datapb.SegmentInfo{ + map[typeutil.UniqueID]*datapb.SegmentInfo{ 100: getSegmentInfo(100, 70000), 101: getSegmentInfo(101, 50000), }, @@ -475,14 +476,14 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { description string segRemained bool - growingSegInfo map[util.UniqueID]*datapb.SegmentInfo + growingSegInfo map[typeutil.UniqueID]*datapb.SegmentInfo inMsg *msgstream.InsertMsg msgFiltered bool }{ { "msgTssegTs", false, - map[util.UniqueID]*datapb.SegmentInfo{ + map[typeutil.UniqueID]*datapb.SegmentInfo{ 100: getSegmentInfo(100, 50000), 101: getSegmentInfo(101, 50000), }, @@ -536,7 +537,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) { func TestFlowGraph_DDNode_isDropped(t *testing.T) { tests := []struct { indroppedSegment []*datapb.SegmentInfo - inSeg util.UniqueID + inSeg typeutil.UniqueID expectedOut bool @@ -582,18 +583,18 @@ func TestFlowGraph_DDNode_isDropped(t *testing.T) { } } -func getSegmentInfo(segmentID util.UniqueID, ts util.Timestamp) *datapb.SegmentInfo { +func getSegmentInfo(segmentID typeutil.UniqueID, ts typeutil.Timestamp) *datapb.SegmentInfo { return &datapb.SegmentInfo{ ID: segmentID, DmlPosition: &msgpb.MsgPosition{Timestamp: ts}, } } -func getInsertMsg(segmentID util.UniqueID, ts util.Timestamp) *msgstream.InsertMsg { +func getInsertMsg(segmentID typeutil.UniqueID, ts typeutil.Timestamp) *msgstream.InsertMsg { return getInsertMsgWithChannel(segmentID, ts, ddNodeChannelName) } -func getInsertMsgWithChannel(segmentID util.UniqueID, ts util.Timestamp, vChannelName string) *msgstream.InsertMsg { +func getInsertMsgWithChannel(segmentID typeutil.UniqueID, ts typeutil.Timestamp, vChannelName string) *msgstream.InsertMsg { return &msgstream.InsertMsg{ BaseMsg: msgstream.BaseMsg{EndTimestamp: ts}, InsertRequest: &msgpb.InsertRequest{ diff --git a/internal/flushcommon/pipeline/flow_graph_dmstream_input_node.go b/internal/flushcommon/pipeline/flow_graph_dmstream_input_node.go index 6207c8ad49..d035592495 100644 --- a/internal/flushcommon/pipeline/flow_graph_dmstream_input_node.go +++ b/internal/flushcommon/pipeline/flow_graph_dmstream_input_node.go @@ -39,27 +39,28 @@ import ( // // messages between two timeticks to the following flowgraph node. In DataNode, the following flow graph node is // flowgraph ddNode. -func newDmInputNode(initCtx context.Context, dispatcherClient msgdispatcher.Client, seekPos *msgpb.MsgPosition, dmNodeConfig *nodeConfig) (*flowgraph.InputNode, error) { +func newDmInputNode(initCtx context.Context, dispatcherClient msgdispatcher.Client, seekPos *msgpb.MsgPosition, dmNodeConfig *nodeConfig, input <-chan *msgstream.MsgPack) (*flowgraph.InputNode, error) { log := log.With(zap.Int64("nodeID", paramtable.GetNodeID()), zap.Int64("collectionID", dmNodeConfig.collectionID), zap.String("vchannel", dmNodeConfig.vChannelName)) var err error - var input <-chan *msgstream.MsgPack - if seekPos != nil && len(seekPos.MsgID) != 0 { - input, err = dispatcherClient.Register(initCtx, dmNodeConfig.vChannelName, seekPos, common.SubscriptionPositionUnknown) - if err != nil { - return nil, err + if input == nil { + if seekPos != nil && len(seekPos.MsgID) != 0 { + input, err = dispatcherClient.Register(initCtx, dmNodeConfig.vChannelName, seekPos, common.SubscriptionPositionUnknown) + if err != nil { + return nil, err + } + log.Info("datanode seek successfully when register to msgDispatcher", + zap.ByteString("msgID", seekPos.GetMsgID()), + zap.Time("tsTime", tsoutil.PhysicalTime(seekPos.GetTimestamp())), + zap.Duration("tsLag", time.Since(tsoutil.PhysicalTime(seekPos.GetTimestamp())))) + } else { + input, err = dispatcherClient.Register(initCtx, dmNodeConfig.vChannelName, nil, common.SubscriptionPositionEarliest) + if err != nil { + return nil, err + } + log.Info("datanode consume successfully when register to msgDispatcher") } - log.Info("datanode seek successfully when register to msgDispatcher", - zap.ByteString("msgID", seekPos.GetMsgID()), - zap.Time("tsTime", tsoutil.PhysicalTime(seekPos.GetTimestamp())), - zap.Duration("tsLag", time.Since(tsoutil.PhysicalTime(seekPos.GetTimestamp())))) - } else { - input, err = dispatcherClient.Register(initCtx, dmNodeConfig.vChannelName, nil, common.SubscriptionPositionEarliest) - if err != nil { - return nil, err - } - log.Info("datanode consume successfully when register to msgDispatcher") } name := fmt.Sprintf("dmInputNode-data-%s", dmNodeConfig.vChannelName) diff --git a/internal/flushcommon/pipeline/flow_graph_dmstream_input_node_test.go b/internal/flushcommon/pipeline/flow_graph_dmstream_input_node_test.go index e6afab0df8..0c0782fcb8 100644 --- a/internal/flushcommon/pipeline/flow_graph_dmstream_input_node_test.go +++ b/internal/flushcommon/pipeline/flow_graph_dmstream_input_node_test.go @@ -111,6 +111,6 @@ func TestNewDmInputNode(t *testing.T) { _, err := newDmInputNode(context.Background(), client, new(msgpb.MsgPosition), &nodeConfig{ msFactory: &mockMsgStreamFactory{}, vChannelName: "mock_vchannel_0", - }) + }, nil) assert.NoError(t, err) } diff --git a/internal/flushcommon/pipeline/flow_graph_manager.go b/internal/flushcommon/pipeline/flow_graph_manager.go index 659ddc3654..2d8b930442 100644 --- a/internal/flushcommon/pipeline/flow_graph_manager.go +++ b/internal/flushcommon/pipeline/flow_graph_manager.go @@ -22,7 +22,7 @@ import ( "go.uber.org/zap" - "github.com/milvus-io/milvus/internal/datanode/util" + "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/util/paramtable" @@ -71,7 +71,7 @@ func (fm *fgManagerImpl) RemoveFlowgraph(channel string) { fm.flowgraphs.Remove(channel) metrics.DataNodeNumFlowGraphs.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Dec() - util.RateCol.RemoveFlowGraphChannel(channel) + util.GetRateCollector().RemoveFlowGraphChannel(channel) } } @@ -95,7 +95,7 @@ func (fm *fgManagerImpl) HasFlowgraph(channel string) bool { return exist } -func (fm *fgManagerImpl) HasFlowgraphWithOpID(channel string, opID util.UniqueID) bool { +func (fm *fgManagerImpl) HasFlowgraphWithOpID(channel string, opID typeutil.UniqueID) bool { ds, exist := fm.flowgraphs.Get(channel) return exist && ds.opID == opID } diff --git a/internal/flushcommon/pipeline/flow_graph_manager_test.go b/internal/flushcommon/pipeline/flow_graph_manager_test.go index 457f09ebb5..aaa25cb4f5 100644 --- a/internal/flushcommon/pipeline/flow_graph_manager_test.go +++ b/internal/flushcommon/pipeline/flow_graph_manager_test.go @@ -29,9 +29,9 @@ 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/allocator" - "github.com/milvus-io/milvus/internal/datanode/broker" - "github.com/milvus-io/milvus/internal/datanode/util" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" + "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/util/sessionutil" @@ -43,10 +43,6 @@ import ( 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) } diff --git a/internal/flushcommon/pipeline/flow_graph_message.go b/internal/flushcommon/pipeline/flow_graph_message.go index ca2b72765e..1222cfd270 100644 --- a/internal/flushcommon/pipeline/flow_graph_message.go +++ b/internal/flushcommon/pipeline/flow_graph_message.go @@ -18,10 +18,11 @@ 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/flushcommon/util" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) type ( @@ -52,12 +53,12 @@ type FlowGraphMsg struct { EndPositions []*msgpb.MsgPosition // segmentsToSync is the signal used by insertBufferNode to notify deleteNode to flush - segmentsToSync []util.UniqueID + segmentsToSync []typeutil.UniqueID dropCollection bool - dropPartitions []util.UniqueID + dropPartitions []typeutil.UniqueID } -func (fgMsg *FlowGraphMsg) TimeTick() util.Timestamp { +func (fgMsg *FlowGraphMsg) TimeTick() typeutil.Timestamp { return fgMsg.TimeRange.TimestampMax } diff --git a/internal/flushcommon/pipeline/flow_graph_message_test.go b/internal/flushcommon/pipeline/flow_graph_message_test.go index 74e2f387ad..30dac9708f 100644 --- a/internal/flushcommon/pipeline/flow_graph_message_test.go +++ b/internal/flushcommon/pipeline/flow_graph_message_test.go @@ -21,12 +21,13 @@ import ( "github.com/stretchr/testify/assert" - "github.com/milvus-io/milvus/internal/datanode/util" + "github.com/milvus-io/milvus/internal/flushcommon/util" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) func TestInsertMsg_TimeTick(te *testing.T) { tests := []struct { - timeTimestanpMax util.Timestamp + timeTimestanpMax typeutil.Timestamp description string }{ diff --git a/internal/flushcommon/pipeline/flow_graph_time_tick_node.go b/internal/flushcommon/pipeline/flow_graph_time_tick_node.go index e1ac9ae708..1fcfeb242f 100644 --- a/internal/flushcommon/pipeline/flow_graph_time_tick_node.go +++ b/internal/flushcommon/pipeline/flow_graph_time_tick_node.go @@ -25,8 +25,8 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/flushcommon/metacache" + "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" "github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/pkg/log" diff --git a/internal/flushcommon/pipeline/flow_graph_write_node.go b/internal/flushcommon/pipeline/flow_graph_write_node.go index 43870c143c..9a8a5ed066 100644 --- a/internal/flushcommon/pipeline/flow_graph_write_node.go +++ b/internal/flushcommon/pipeline/flow_graph_write_node.go @@ -11,8 +11,8 @@ 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/internal/flushcommon/metacache" + "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" diff --git a/internal/datanode/util/testutils.go b/internal/flushcommon/pipeline/testutils_test.go similarity index 94% rename from internal/datanode/util/testutils.go rename to internal/flushcommon/pipeline/testutils_test.go index 98b09b0f7e..cf2ab1e351 100644 --- a/internal/datanode/util/testutils.go +++ b/internal/flushcommon/pipeline/testutils_test.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package util +package pipeline import ( "bytes" @@ -42,6 +42,7 @@ import ( "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" "github.com/milvus-io/milvus/pkg/util/merr" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) const returnError = "ReturnError" @@ -81,9 +82,9 @@ type DataFactory struct { type RootCoordFactory struct { types.RootCoordClient - ID UniqueID + ID typeutil.UniqueID collectionName string - collectionID UniqueID + collectionID typeutil.UniqueID pkType schemapb.DataType ReportImportErr bool @@ -235,7 +236,7 @@ func (ds *DataCoordFactory) GetSegmentInfo(ctx context.Context, req *datapb.GetS }, nil } -func (mf *MetaFactory) GetCollectionMeta(collectionID UniqueID, collectionName string, pkDataType schemapb.DataType) *etcdpb.CollectionMeta { +func (mf *MetaFactory) GetCollectionMeta(collectionID typeutil.UniqueID, collectionName string, pkDataType schemapb.DataType) *etcdpb.CollectionMeta { sch := schemapb.CollectionSchema{ Name: collectionName, Description: "test collection by meta factory", @@ -251,9 +252,9 @@ func (mf *MetaFactory) GetCollectionMeta(collectionID UniqueID, collectionName s return &etcdpb.CollectionMeta{ ID: collectionID, Schema: &sch, - CreateTime: Timestamp(1), - SegmentIDs: make([]UniqueID, 0), - PartitionIDs: []UniqueID{0}, + CreateTime: typeutil.Timestamp(1), + SegmentIDs: make([]typeutil.UniqueID, 0), + PartitionIDs: []typeutil.UniqueID{0}, } } @@ -669,16 +670,16 @@ func (df *DataFactory) GenMsgStreamInsertMsg(idx int, chanName string) *msgstrea Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_Insert, MsgID: 0, - Timestamp: Timestamp(idx + 1000), + Timestamp: typeutil.Timestamp(idx + 1000), SourceID: 0, }, CollectionName: "col1", PartitionName: "default", SegmentID: 1, - CollectionID: UniqueID(0), + CollectionID: typeutil.UniqueID(0), ShardName: chanName, - Timestamps: []Timestamp{Timestamp(idx + 1000)}, - RowIDs: []UniqueID{UniqueID(idx)}, + Timestamps: []typeutil.Timestamp{typeutil.Timestamp(idx + 1000)}, + RowIDs: []typeutil.UniqueID{typeutil.UniqueID(idx)}, // RowData: []*commonpb.Blob{{Value: df.rawData}}, FieldsData: df.columnData, Version: msgpb.InsertDataVersion_ColumnBased, @@ -688,7 +689,7 @@ func (df *DataFactory) GenMsgStreamInsertMsg(idx int, chanName string) *msgstrea return msg } -func (df *DataFactory) GenMsgStreamInsertMsgWithTs(idx int, chanName string, ts Timestamp) *msgstream.InsertMsg { +func (df *DataFactory) GenMsgStreamInsertMsgWithTs(idx int, chanName string, ts typeutil.Timestamp) *msgstream.InsertMsg { msg := &msgstream.InsertMsg{ BaseMsg: msgstream.BaseMsg{ HashValues: []uint32{uint32(idx)}, @@ -705,10 +706,10 @@ func (df *DataFactory) GenMsgStreamInsertMsgWithTs(idx int, chanName string, ts CollectionName: "col1", PartitionName: "default", SegmentID: 1, - CollectionID: UniqueID(0), + CollectionID: typeutil.UniqueID(0), ShardName: chanName, - Timestamps: []Timestamp{ts}, - RowIDs: []UniqueID{UniqueID(idx)}, + Timestamps: []typeutil.Timestamp{ts}, + RowIDs: []typeutil.UniqueID{typeutil.UniqueID(idx)}, // RowData: []*commonpb.Blob{{Value: df.rawData}}, FieldsData: df.columnData, Version: msgpb.InsertDataVersion_ColumnBased, @@ -718,7 +719,7 @@ func (df *DataFactory) GenMsgStreamInsertMsgWithTs(idx int, chanName string, ts return msg } -func (df *DataFactory) GetMsgStreamTsInsertMsgs(n int, chanName string, ts Timestamp) (inMsgs []msgstream.TsMsg) { +func (df *DataFactory) GetMsgStreamTsInsertMsgs(n int, chanName string, ts typeutil.Timestamp) (inMsgs []msgstream.TsMsg) { for i := 0; i < n; i++ { msg := df.GenMsgStreamInsertMsgWithTs(i, chanName, ts) var tsMsg msgstream.TsMsg = msg @@ -737,9 +738,9 @@ func (df *DataFactory) GetMsgStreamInsertMsgs(n int) (msgs []*msgstream.InsertMs func (df *DataFactory) GenMsgStreamDeleteMsg(pks []storage.PrimaryKey, chanName string) *msgstream.DeleteMsg { idx := 100 - timestamps := make([]Timestamp, len(pks)) + timestamps := make([]typeutil.Timestamp, len(pks)) for i := 0; i < len(pks); i++ { - timestamps[i] = Timestamp(i) + 1000 + timestamps[i] = typeutil.Timestamp(i) + 1000 } msg := &msgstream.DeleteMsg{ BaseMsg: msgstream.BaseMsg{ @@ -749,7 +750,7 @@ func (df *DataFactory) GenMsgStreamDeleteMsg(pks []storage.PrimaryKey, chanName Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_Delete, MsgID: 0, - Timestamp: Timestamp(idx + 1000), + Timestamp: typeutil.Timestamp(idx + 1000), SourceID: 0, }, CollectionName: "col1", @@ -764,7 +765,7 @@ func (df *DataFactory) GenMsgStreamDeleteMsg(pks []storage.PrimaryKey, chanName return msg } -func (df *DataFactory) GenMsgStreamDeleteMsgWithTs(idx int, pks []storage.PrimaryKey, chanName string, ts Timestamp) *msgstream.DeleteMsg { +func (df *DataFactory) GenMsgStreamDeleteMsgWithTs(idx int, pks []storage.PrimaryKey, chanName string, ts typeutil.Timestamp) *msgstream.DeleteMsg { msg := &msgstream.DeleteMsg{ BaseMsg: msgstream.BaseMsg{ HashValues: []uint32{uint32(idx)}, @@ -781,17 +782,17 @@ func (df *DataFactory) GenMsgStreamDeleteMsgWithTs(idx int, pks []storage.Primar CollectionName: "col1", PartitionName: "default", PartitionID: 1, - CollectionID: UniqueID(0), + CollectionID: typeutil.UniqueID(0), ShardName: chanName, PrimaryKeys: storage.ParsePrimaryKeys2IDs(pks), - Timestamps: []Timestamp{ts}, + Timestamps: []typeutil.Timestamp{ts}, NumRows: int64(len(pks)), }, } return msg } -func (m *RootCoordFactory) setCollectionID(id UniqueID) { +func (m *RootCoordFactory) setCollectionID(id typeutil.UniqueID) { m.collectionID = id } @@ -942,7 +943,7 @@ func EmptyBfsFactory(info *datapb.SegmentInfo) *metacache.BloomFilterSet { return metacache.NewBloomFilterSet() } -func GetWatchInfoByOpID(opID UniqueID, channel string, state datapb.ChannelWatchState) *datapb.ChannelWatchInfo { +func GetWatchInfoByOpID(opID typeutil.UniqueID, channel string, state datapb.ChannelWatchState) *datapb.ChannelWatchInfo { return &datapb.ChannelWatchInfo{ OpID: opID, State: state, diff --git a/internal/flushcommon/syncmgr/meta_writer.go b/internal/flushcommon/syncmgr/meta_writer.go index 19853f329d..f558a76717 100644 --- a/internal/flushcommon/syncmgr/meta_writer.go +++ b/internal/flushcommon/syncmgr/meta_writer.go @@ -8,7 +8,7 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/datanode/broker" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" diff --git a/internal/flushcommon/syncmgr/meta_writer_test.go b/internal/flushcommon/syncmgr/meta_writer_test.go index 479b9fb375..d74d9d464a 100644 --- a/internal/flushcommon/syncmgr/meta_writer_test.go +++ b/internal/flushcommon/syncmgr/meta_writer_test.go @@ -8,7 +8,7 @@ import ( "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" - "github.com/milvus-io/milvus/internal/datanode/broker" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/util/paramtable" diff --git a/internal/flushcommon/syncmgr/sync_manager.go b/internal/flushcommon/syncmgr/sync_manager.go index cac7d76827..329e354e36 100644 --- a/internal/flushcommon/syncmgr/sync_manager.go +++ b/internal/flushcommon/syncmgr/sync_manager.go @@ -15,7 +15,6 @@ import ( "github.com/milvus-io/milvus/pkg/config" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/conc" - "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -55,12 +54,9 @@ type syncManager struct { tasks *typeutil.ConcurrentMap[string, Task] } -func NewSyncManager(chunkManager storage.ChunkManager) (SyncManager, error) { +func NewSyncManager(chunkManager storage.ChunkManager) SyncManager { params := paramtable.Get() initPoolSize := params.DataNodeCfg.MaxParallelSyncMgrTasks.GetAsInt() - if initPoolSize < 1 { - return nil, merr.WrapErrParameterInvalid("positive parallel task number", strconv.FormatInt(int64(initPoolSize), 10)) - } dispatcher := newKeyLockDispatcher[int64](initPoolSize) log.Info("sync manager initialized", zap.Int("initPoolSize", initPoolSize)) @@ -71,8 +67,7 @@ func NewSyncManager(chunkManager storage.ChunkManager) (SyncManager, error) { } // setup config update watcher params.Watch(params.DataNodeCfg.MaxParallelSyncMgrTasks.Key, config.NewHandler("datanode.syncmgr.poolsize", syncMgr.resizeHandler)) - - return syncMgr, nil + return syncMgr } func (mgr *syncManager) resizeHandler(evt *config.Event) { diff --git a/internal/flushcommon/syncmgr/sync_manager_test.go b/internal/flushcommon/syncmgr/sync_manager_test.go index 083a29ec20..0267a47328 100644 --- a/internal/flushcommon/syncmgr/sync_manager_test.go +++ b/internal/flushcommon/syncmgr/sync_manager_test.go @@ -17,7 +17,7 @@ import ( "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/allocator" - "github.com/milvus-io/milvus/internal/datanode/broker" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/proto/datapb" @@ -160,8 +160,7 @@ func (s *SyncManagerSuite) TestSubmit() { s.metacache.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg}) s.metacache.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return() - manager, err := NewSyncManager(s.chunkManager) - s.NoError(err) + manager := NewSyncManager(s.chunkManager) task := s.getSuiteSyncTask() task.WithMetaWriter(BrokerMetaWriter(s.broker, 1)) task.WithTimeRange(50, 100) @@ -174,7 +173,7 @@ func (s *SyncManagerSuite) TestSubmit() { f := manager.SyncData(context.Background(), task) s.NotNil(f) - _, err = f.Await() + _, err := f.Await() s.NoError(err) } @@ -190,8 +189,7 @@ func (s *SyncManagerSuite) TestCompacted() { s.metacache.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg}) s.metacache.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return() - manager, err := NewSyncManager(s.chunkManager) - s.NoError(err) + manager := NewSyncManager(s.chunkManager) task := s.getSuiteSyncTask() task.WithMetaWriter(BrokerMetaWriter(s.broker, 1)) task.WithTimeRange(50, 100) @@ -204,14 +202,13 @@ func (s *SyncManagerSuite) TestCompacted() { f := manager.SyncData(context.Background(), task) s.NotNil(f) - _, err = f.Await() + _, err := f.Await() s.NoError(err) s.EqualValues(1001, segmentID.Load()) } func (s *SyncManagerSuite) TestResizePool() { - manager, err := NewSyncManager(s.chunkManager) - s.NoError(err) + manager := NewSyncManager(s.chunkManager) syncMgr, ok := manager.(*syncManager) s.Require().True(ok) @@ -245,26 +242,8 @@ func (s *SyncManagerSuite) TestResizePool() { s.Equal(cap*2, syncMgr.keyLockDispatcher.workerPool.Cap()) } -func (s *SyncManagerSuite) TestNewSyncManager() { - manager, err := NewSyncManager(s.chunkManager) - s.NoError(err) - - _, ok := manager.(*syncManager) - s.Require().True(ok) - - params := paramtable.Get() - configKey := params.DataNodeCfg.MaxParallelSyncMgrTasks.Key - defer params.Reset(configKey) - - params.Save(configKey, "0") - - _, err = NewSyncManager(s.chunkManager) - s.Error(err) -} - func (s *SyncManagerSuite) TestUnexpectedError() { - manager, err := NewSyncManager(s.chunkManager) - s.NoError(err) + manager := NewSyncManager(s.chunkManager) task := NewMockTask(s.T()) task.EXPECT().SegmentID().Return(1000) @@ -273,13 +252,12 @@ func (s *SyncManagerSuite) TestUnexpectedError() { task.EXPECT().HandleError(mock.Anything) f := manager.SyncData(context.Background(), task) - _, err = f.Await() + _, err := f.Await() s.Error(err) } func (s *SyncManagerSuite) TestTargetUpdateSameID() { - manager, err := NewSyncManager(s.chunkManager) - s.NoError(err) + manager := NewSyncManager(s.chunkManager) task := NewMockTask(s.T()) task.EXPECT().SegmentID().Return(1000) @@ -288,7 +266,7 @@ func (s *SyncManagerSuite) TestTargetUpdateSameID() { task.EXPECT().HandleError(mock.Anything) f := manager.SyncData(context.Background(), task) - _, err = f.Await() + _, err := f.Await() s.Error(err) } diff --git a/internal/flushcommon/syncmgr/task_test.go b/internal/flushcommon/syncmgr/task_test.go index ffb87252b9..7ce1d78d7d 100644 --- a/internal/flushcommon/syncmgr/task_test.go +++ b/internal/flushcommon/syncmgr/task_test.go @@ -32,7 +32,7 @@ import ( "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/allocator" - "github.com/milvus-io/milvus/internal/datanode/broker" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/proto/datapb" diff --git a/internal/datanode/util/checkpoint_updater.go b/internal/flushcommon/util/checkpoint_updater.go similarity index 99% rename from internal/datanode/util/checkpoint_updater.go rename to internal/flushcommon/util/checkpoint_updater.go index 99d70c5be6..302f44db7d 100644 --- a/internal/datanode/util/checkpoint_updater.go +++ b/internal/flushcommon/util/checkpoint_updater.go @@ -25,7 +25,7 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/datanode/broker" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" diff --git a/internal/datanode/util/checkpoint_updater_test.go b/internal/flushcommon/util/checkpoint_updater_test.go similarity index 96% rename from internal/datanode/util/checkpoint_updater_test.go rename to internal/flushcommon/util/checkpoint_updater_test.go index 7e75d588b9..a7266f8de0 100644 --- a/internal/datanode/util/checkpoint_updater_test.go +++ b/internal/flushcommon/util/checkpoint_updater_test.go @@ -28,7 +28,7 @@ import ( "go.uber.org/atomic" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" - "github.com/milvus-io/milvus/internal/datanode/broker" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/pkg/util/paramtable" ) @@ -40,6 +40,7 @@ type ChannelCPUpdaterSuite struct { } func (s *ChannelCPUpdaterSuite) SetupTest() { + paramtable.Init() s.broker = broker.NewMockBroker(s.T()) s.updater = NewChannelCheckpointUpdater(s.broker) } diff --git a/internal/datanode/util/rate_collector.go b/internal/flushcommon/util/rate_collector.go similarity index 71% rename from internal/datanode/util/rate_collector.go rename to internal/flushcommon/util/rate_collector.go index f7fcd886ae..4736eb2209 100644 --- a/internal/datanode/util/rate_collector.go +++ b/internal/flushcommon/util/rate_collector.go @@ -19,14 +19,17 @@ package util import ( "sync" + "go.uber.org/zap" + + "github.com/milvus-io/milvus/pkg/log" "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 ) @@ -35,41 +38,49 @@ type RateCollector struct { *ratelimitutil.RateCollector flowGraphTtMu sync.Mutex - flowGraphTt map[string]Timestamp + flowGraphTt map[string]typeutil.Timestamp } -func InitGlobalRateCollector() error { - var err error +func initGlobalRateCollector() { initOnce.Do(func() { - RateCol, err = NewRateCollector() + var err error + rateCol, err = newRateCollector() + if err != nil { + log.Warn("DataNode server init rateCollector failed", zap.Error(err)) + panic(err) + } + rateCol.Register(metricsinfo.InsertConsumeThroughput) + rateCol.Register(metricsinfo.DeleteConsumeThroughput) }) - RateCol.Register(metricsinfo.InsertConsumeThroughput) - RateCol.Register(metricsinfo.DeleteConsumeThroughput) - return err } func DeregisterRateCollector(label string) { - RateCol.Deregister(label) + rateCol.Deregister(label) } func RegisterRateCollector(label string) { - RateCol.Register(label) + rateCol.Register(label) +} + +func GetRateCollector() *RateCollector { + initGlobalRateCollector() + return rateCol } // newRateCollector returns a new RateCollector. -func NewRateCollector() (*RateCollector, error) { +func newRateCollector() (*RateCollector, error) { rc, err := ratelimitutil.NewRateCollector(ratelimitutil.DefaultWindow, ratelimitutil.DefaultGranularity, false) if err != nil { return nil, err } return &RateCollector{ RateCollector: rc, - flowGraphTt: make(map[string]Timestamp), + flowGraphTt: make(map[string]typeutil.Timestamp), }, nil } // UpdateFlowGraphTt updates RateCollector's flow graph time tick. -func (r *RateCollector) UpdateFlowGraphTt(channel string, t Timestamp) { +func (r *RateCollector) UpdateFlowGraphTt(channel string, t typeutil.Timestamp) { r.flowGraphTtMu.Lock() defer r.flowGraphTtMu.Unlock() r.flowGraphTt[channel] = t @@ -83,7 +94,7 @@ func (r *RateCollector) RemoveFlowGraphChannel(channel string) { } // GetMinFlowGraphTt returns the vchannel and minimal time tick of flow graphs. -func (r *RateCollector) GetMinFlowGraphTt() (string, Timestamp) { +func (r *RateCollector) GetMinFlowGraphTt() (string, typeutil.Timestamp) { r.flowGraphTtMu.Lock() defer r.flowGraphTtMu.Unlock() minTt := typeutil.MaxTimestamp diff --git a/internal/datanode/util/rate_collector_test.go b/internal/flushcommon/util/rate_collector_test.go similarity index 93% rename from internal/datanode/util/rate_collector_test.go rename to internal/flushcommon/util/rate_collector_test.go index e5c8dbe4c1..f672b5869e 100644 --- a/internal/datanode/util/rate_collector_test.go +++ b/internal/flushcommon/util/rate_collector_test.go @@ -26,7 +26,7 @@ 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() @@ -37,6 +37,6 @@ func TestRateCollector(t *testing.T) { collector.UpdateFlowGraphTt("channel3", 50) c, minTt = collector.GetMinFlowGraphTt() assert.Equal(t, "channel3", c) - assert.Equal(t, Timestamp(50), minTt) + assert.Equal(t, typeutil.Timestamp(50), minTt) }) } diff --git a/internal/datanode/util/tickler.go b/internal/flushcommon/util/tickler.go similarity index 100% rename from internal/datanode/util/tickler.go rename to internal/flushcommon/util/tickler.go diff --git a/internal/datanode/util/timetick_sender.go b/internal/flushcommon/util/timetick_sender.go similarity index 96% rename from internal/datanode/util/timetick_sender.go rename to internal/flushcommon/util/timetick_sender.go index 3bceb52457..ad9f824408 100644 --- a/internal/datanode/util/timetick_sender.go +++ b/internal/flushcommon/util/timetick_sender.go @@ -26,15 +26,16 @@ 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/broker" + "github.com/milvus-io/milvus/internal/flushcommon/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" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) type StatsUpdater interface { - Update(channel string, ts Timestamp, stats []*commonpb.SegmentStats) + Update(channel string, ts typeutil.Timestamp, stats []*commonpb.SegmentStats) } // TimeTickSender is to merge channel states updated by flow graph node and send to datacoord periodically diff --git a/internal/datanode/util/timetick_sender_test.go b/internal/flushcommon/util/timetick_sender_test.go similarity index 98% rename from internal/datanode/util/timetick_sender_test.go rename to internal/flushcommon/util/timetick_sender_test.go index b0621c66e1..0edaf29078 100644 --- a/internal/datanode/util/timetick_sender_test.go +++ b/internal/flushcommon/util/timetick_sender_test.go @@ -28,7 +28,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/broker" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/retry" diff --git a/internal/datanode/util/util.go b/internal/flushcommon/util/util.go similarity index 84% rename from internal/datanode/util/util.go rename to internal/flushcommon/util/util.go index 31f960a5a9..cfbe6b4507 100644 --- a/internal/datanode/util/util.go +++ b/internal/flushcommon/util/util.go @@ -22,12 +22,13 @@ 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/allocator" "github.com/milvus-io/milvus/internal/datanode/compaction" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/internal/streamingnode/server/flusher" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" @@ -35,20 +36,6 @@ import ( "github.com/milvus-io/milvus/pkg/util/typeutil" ) -type ( - // UniqueID is type int64 - UniqueID = typeutil.UniqueID - - // Timestamp is type uint64 - Timestamp = typeutil.Timestamp - - // IntPrimaryKey is type int64 - IntPrimaryKey = typeutil.IntPrimaryKey - - // DSL is type string - DSL = string -) - type PipelineParams struct { Ctx context.Context Broker broker.Broker @@ -61,13 +48,14 @@ type PipelineParams struct { Session *sessionutil.Session WriteBufferManager writebuffer.BufferManager CheckpointUpdater *ChannelCheckpointUpdater - Allocator allocator.Allocator + Allocator allocator.Interface + FlushMsgHandler flusher.FlushMsgHandler } // TimeRange is a range of timestamp contains the min-timestamp and max-timestamp type TimeRange struct { - TimestampMin Timestamp - TimestampMax Timestamp + TimestampMin typeutil.Timestamp + TimestampMax typeutil.Timestamp } func StartTracer(msg msgstream.TsMsg, name string) (context.Context, trace.Span) { diff --git a/internal/flushcommon/writebuffer/bf_write_buffer_test.go b/internal/flushcommon/writebuffer/bf_write_buffer_test.go index b8ec5a0c23..bea8856efa 100644 --- a/internal/flushcommon/writebuffer/bf_write_buffer_test.go +++ b/internal/flushcommon/writebuffer/bf_write_buffer_test.go @@ -13,7 +13,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-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/datanode/broker" + "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" "github.com/milvus-io/milvus/internal/proto/datapb" diff --git a/internal/flushcommon/writebuffer/l0_write_buffer.go b/internal/flushcommon/writebuffer/l0_write_buffer.go index 67647a6255..7fbc376dc0 100644 --- a/internal/flushcommon/writebuffer/l0_write_buffer.go +++ b/internal/flushcommon/writebuffer/l0_write_buffer.go @@ -9,7 +9,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/allocator" - "github.com/milvus-io/milvus/internal/datanode/io" + "github.com/milvus-io/milvus/internal/flushcommon/io" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" "github.com/milvus-io/milvus/internal/proto/datapb" diff --git a/internal/mocks/mock_datacoord.go b/internal/mocks/mock_datacoord.go index 24e7b8e248..77c39cd18e 100644 --- a/internal/mocks/mock_datacoord.go +++ b/internal/mocks/mock_datacoord.go @@ -125,8 +125,8 @@ type MockDataCoord_AlterIndex_Call struct { } // AlterIndex is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *indexpb.AlterIndexRequest +// - _a0 context.Context +// - _a1 *indexpb.AlterIndexRequest func (_e *MockDataCoord_Expecter) AlterIndex(_a0 interface{}, _a1 interface{}) *MockDataCoord_AlterIndex_Call { return &MockDataCoord_AlterIndex_Call{Call: _e.mock.On("AlterIndex", _a0, _a1)} } @@ -180,8 +180,8 @@ type MockDataCoord_AssignSegmentID_Call struct { } // AssignSegmentID is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.AssignSegmentIDRequest +// - _a0 context.Context +// - _a1 *datapb.AssignSegmentIDRequest func (_e *MockDataCoord_Expecter) AssignSegmentID(_a0 interface{}, _a1 interface{}) *MockDataCoord_AssignSegmentID_Call { return &MockDataCoord_AssignSegmentID_Call{Call: _e.mock.On("AssignSegmentID", _a0, _a1)} } @@ -235,8 +235,8 @@ type MockDataCoord_BroadcastAlteredCollection_Call struct { } // BroadcastAlteredCollection is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.AlterCollectionRequest +// - _a0 context.Context +// - _a1 *datapb.AlterCollectionRequest func (_e *MockDataCoord_Expecter) BroadcastAlteredCollection(_a0 interface{}, _a1 interface{}) *MockDataCoord_BroadcastAlteredCollection_Call { return &MockDataCoord_BroadcastAlteredCollection_Call{Call: _e.mock.On("BroadcastAlteredCollection", _a0, _a1)} } @@ -290,8 +290,8 @@ type MockDataCoord_CheckHealth_Call struct { } // CheckHealth is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *milvuspb.CheckHealthRequest +// - _a0 context.Context +// - _a1 *milvuspb.CheckHealthRequest func (_e *MockDataCoord_Expecter) CheckHealth(_a0 interface{}, _a1 interface{}) *MockDataCoord_CheckHealth_Call { return &MockDataCoord_CheckHealth_Call{Call: _e.mock.On("CheckHealth", _a0, _a1)} } @@ -345,8 +345,8 @@ type MockDataCoord_CreateIndex_Call struct { } // CreateIndex is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *indexpb.CreateIndexRequest +// - _a0 context.Context +// - _a1 *indexpb.CreateIndexRequest func (_e *MockDataCoord_Expecter) CreateIndex(_a0 interface{}, _a1 interface{}) *MockDataCoord_CreateIndex_Call { return &MockDataCoord_CreateIndex_Call{Call: _e.mock.On("CreateIndex", _a0, _a1)} } @@ -400,8 +400,8 @@ type MockDataCoord_DescribeIndex_Call struct { } // DescribeIndex is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *indexpb.DescribeIndexRequest +// - _a0 context.Context +// - _a1 *indexpb.DescribeIndexRequest func (_e *MockDataCoord_Expecter) DescribeIndex(_a0 interface{}, _a1 interface{}) *MockDataCoord_DescribeIndex_Call { return &MockDataCoord_DescribeIndex_Call{Call: _e.mock.On("DescribeIndex", _a0, _a1)} } @@ -455,8 +455,8 @@ type MockDataCoord_DropIndex_Call struct { } // DropIndex is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *indexpb.DropIndexRequest +// - _a0 context.Context +// - _a1 *indexpb.DropIndexRequest func (_e *MockDataCoord_Expecter) DropIndex(_a0 interface{}, _a1 interface{}) *MockDataCoord_DropIndex_Call { return &MockDataCoord_DropIndex_Call{Call: _e.mock.On("DropIndex", _a0, _a1)} } @@ -510,8 +510,8 @@ type MockDataCoord_DropVirtualChannel_Call struct { } // DropVirtualChannel is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.DropVirtualChannelRequest +// - _a0 context.Context +// - _a1 *datapb.DropVirtualChannelRequest func (_e *MockDataCoord_Expecter) DropVirtualChannel(_a0 interface{}, _a1 interface{}) *MockDataCoord_DropVirtualChannel_Call { return &MockDataCoord_DropVirtualChannel_Call{Call: _e.mock.On("DropVirtualChannel", _a0, _a1)} } @@ -565,8 +565,8 @@ type MockDataCoord_Flush_Call struct { } // Flush is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.FlushRequest +// - _a0 context.Context +// - _a1 *datapb.FlushRequest func (_e *MockDataCoord_Expecter) Flush(_a0 interface{}, _a1 interface{}) *MockDataCoord_Flush_Call { return &MockDataCoord_Flush_Call{Call: _e.mock.On("Flush", _a0, _a1)} } @@ -620,8 +620,8 @@ type MockDataCoord_GcConfirm_Call struct { } // GcConfirm is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.GcConfirmRequest +// - _a0 context.Context +// - _a1 *datapb.GcConfirmRequest func (_e *MockDataCoord_Expecter) GcConfirm(_a0 interface{}, _a1 interface{}) *MockDataCoord_GcConfirm_Call { return &MockDataCoord_GcConfirm_Call{Call: _e.mock.On("GcConfirm", _a0, _a1)} } @@ -675,8 +675,8 @@ type MockDataCoord_GcControl_Call struct { } // GcControl is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.GcControlRequest +// - _a0 context.Context +// - _a1 *datapb.GcControlRequest func (_e *MockDataCoord_Expecter) GcControl(_a0 interface{}, _a1 interface{}) *MockDataCoord_GcControl_Call { return &MockDataCoord_GcControl_Call{Call: _e.mock.On("GcControl", _a0, _a1)} } @@ -698,6 +698,61 @@ func (_c *MockDataCoord_GcControl_Call) RunAndReturn(run func(context.Context, * return _c } +// GetChannelRecoveryInfo provides a mock function with given fields: _a0, _a1 +func (_m *MockDataCoord) GetChannelRecoveryInfo(_a0 context.Context, _a1 *datapb.GetChannelRecoveryInfoRequest) (*datapb.GetChannelRecoveryInfoResponse, error) { + ret := _m.Called(_a0, _a1) + + var r0 *datapb.GetChannelRecoveryInfoResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *datapb.GetChannelRecoveryInfoRequest) (*datapb.GetChannelRecoveryInfoResponse, error)); ok { + return rf(_a0, _a1) + } + if rf, ok := ret.Get(0).(func(context.Context, *datapb.GetChannelRecoveryInfoRequest) *datapb.GetChannelRecoveryInfoResponse); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*datapb.GetChannelRecoveryInfoResponse) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *datapb.GetChannelRecoveryInfoRequest) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockDataCoord_GetChannelRecoveryInfo_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetChannelRecoveryInfo' +type MockDataCoord_GetChannelRecoveryInfo_Call struct { + *mock.Call +} + +// GetChannelRecoveryInfo is a helper method to define mock.On call +// - _a0 context.Context +// - _a1 *datapb.GetChannelRecoveryInfoRequest +func (_e *MockDataCoord_Expecter) GetChannelRecoveryInfo(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetChannelRecoveryInfo_Call { + return &MockDataCoord_GetChannelRecoveryInfo_Call{Call: _e.mock.On("GetChannelRecoveryInfo", _a0, _a1)} +} + +func (_c *MockDataCoord_GetChannelRecoveryInfo_Call) Run(run func(_a0 context.Context, _a1 *datapb.GetChannelRecoveryInfoRequest)) *MockDataCoord_GetChannelRecoveryInfo_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*datapb.GetChannelRecoveryInfoRequest)) + }) + return _c +} + +func (_c *MockDataCoord_GetChannelRecoveryInfo_Call) Return(_a0 *datapb.GetChannelRecoveryInfoResponse, _a1 error) *MockDataCoord_GetChannelRecoveryInfo_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockDataCoord_GetChannelRecoveryInfo_Call) RunAndReturn(run func(context.Context, *datapb.GetChannelRecoveryInfoRequest) (*datapb.GetChannelRecoveryInfoResponse, error)) *MockDataCoord_GetChannelRecoveryInfo_Call { + _c.Call.Return(run) + return _c +} + // GetCollectionStatistics provides a mock function with given fields: _a0, _a1 func (_m *MockDataCoord) GetCollectionStatistics(_a0 context.Context, _a1 *datapb.GetCollectionStatisticsRequest) (*datapb.GetCollectionStatisticsResponse, error) { ret := _m.Called(_a0, _a1) @@ -730,8 +785,8 @@ type MockDataCoord_GetCollectionStatistics_Call struct { } // GetCollectionStatistics is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.GetCollectionStatisticsRequest +// - _a0 context.Context +// - _a1 *datapb.GetCollectionStatisticsRequest func (_e *MockDataCoord_Expecter) GetCollectionStatistics(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetCollectionStatistics_Call { return &MockDataCoord_GetCollectionStatistics_Call{Call: _e.mock.On("GetCollectionStatistics", _a0, _a1)} } @@ -785,8 +840,8 @@ type MockDataCoord_GetCompactionState_Call struct { } // GetCompactionState is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *milvuspb.GetCompactionStateRequest +// - _a0 context.Context +// - _a1 *milvuspb.GetCompactionStateRequest func (_e *MockDataCoord_Expecter) GetCompactionState(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetCompactionState_Call { return &MockDataCoord_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState", _a0, _a1)} } @@ -840,8 +895,8 @@ type MockDataCoord_GetCompactionStateWithPlans_Call struct { } // GetCompactionStateWithPlans is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *milvuspb.GetCompactionPlansRequest +// - _a0 context.Context +// - _a1 *milvuspb.GetCompactionPlansRequest func (_e *MockDataCoord_Expecter) GetCompactionStateWithPlans(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetCompactionStateWithPlans_Call { return &MockDataCoord_GetCompactionStateWithPlans_Call{Call: _e.mock.On("GetCompactionStateWithPlans", _a0, _a1)} } @@ -895,8 +950,8 @@ type MockDataCoord_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 *MockDataCoord_Expecter) GetComponentStates(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetComponentStates_Call { return &MockDataCoord_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", _a0, _a1)} } @@ -950,8 +1005,8 @@ type MockDataCoord_GetFlushAllState_Call struct { } // GetFlushAllState is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *milvuspb.GetFlushAllStateRequest +// - _a0 context.Context +// - _a1 *milvuspb.GetFlushAllStateRequest func (_e *MockDataCoord_Expecter) GetFlushAllState(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetFlushAllState_Call { return &MockDataCoord_GetFlushAllState_Call{Call: _e.mock.On("GetFlushAllState", _a0, _a1)} } @@ -1005,8 +1060,8 @@ type MockDataCoord_GetFlushState_Call struct { } // GetFlushState is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.GetFlushStateRequest +// - _a0 context.Context +// - _a1 *datapb.GetFlushStateRequest func (_e *MockDataCoord_Expecter) GetFlushState(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetFlushState_Call { return &MockDataCoord_GetFlushState_Call{Call: _e.mock.On("GetFlushState", _a0, _a1)} } @@ -1060,8 +1115,8 @@ type MockDataCoord_GetFlushedSegments_Call struct { } // GetFlushedSegments is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.GetFlushedSegmentsRequest +// - _a0 context.Context +// - _a1 *datapb.GetFlushedSegmentsRequest func (_e *MockDataCoord_Expecter) GetFlushedSegments(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetFlushedSegments_Call { return &MockDataCoord_GetFlushedSegments_Call{Call: _e.mock.On("GetFlushedSegments", _a0, _a1)} } @@ -1115,8 +1170,8 @@ type MockDataCoord_GetImportProgress_Call struct { } // GetImportProgress is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *internalpb.GetImportProgressRequest +// - _a0 context.Context +// - _a1 *internalpb.GetImportProgressRequest func (_e *MockDataCoord_Expecter) GetImportProgress(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetImportProgress_Call { return &MockDataCoord_GetImportProgress_Call{Call: _e.mock.On("GetImportProgress", _a0, _a1)} } @@ -1170,8 +1225,8 @@ type MockDataCoord_GetIndexBuildProgress_Call struct { } // GetIndexBuildProgress is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *indexpb.GetIndexBuildProgressRequest +// - _a0 context.Context +// - _a1 *indexpb.GetIndexBuildProgressRequest func (_e *MockDataCoord_Expecter) GetIndexBuildProgress(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetIndexBuildProgress_Call { return &MockDataCoord_GetIndexBuildProgress_Call{Call: _e.mock.On("GetIndexBuildProgress", _a0, _a1)} } @@ -1225,8 +1280,8 @@ type MockDataCoord_GetIndexInfos_Call struct { } // GetIndexInfos is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *indexpb.GetIndexInfoRequest +// - _a0 context.Context +// - _a1 *indexpb.GetIndexInfoRequest func (_e *MockDataCoord_Expecter) GetIndexInfos(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetIndexInfos_Call { return &MockDataCoord_GetIndexInfos_Call{Call: _e.mock.On("GetIndexInfos", _a0, _a1)} } @@ -1280,8 +1335,8 @@ type MockDataCoord_GetIndexState_Call struct { } // GetIndexState is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *indexpb.GetIndexStateRequest +// - _a0 context.Context +// - _a1 *indexpb.GetIndexStateRequest func (_e *MockDataCoord_Expecter) GetIndexState(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetIndexState_Call { return &MockDataCoord_GetIndexState_Call{Call: _e.mock.On("GetIndexState", _a0, _a1)} } @@ -1335,8 +1390,8 @@ type MockDataCoord_GetIndexStatistics_Call struct { } // GetIndexStatistics is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *indexpb.GetIndexStatisticsRequest +// - _a0 context.Context +// - _a1 *indexpb.GetIndexStatisticsRequest func (_e *MockDataCoord_Expecter) GetIndexStatistics(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetIndexStatistics_Call { return &MockDataCoord_GetIndexStatistics_Call{Call: _e.mock.On("GetIndexStatistics", _a0, _a1)} } @@ -1390,8 +1445,8 @@ type MockDataCoord_GetInsertBinlogPaths_Call struct { } // GetInsertBinlogPaths is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.GetInsertBinlogPathsRequest +// - _a0 context.Context +// - _a1 *datapb.GetInsertBinlogPathsRequest func (_e *MockDataCoord_Expecter) GetInsertBinlogPaths(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetInsertBinlogPaths_Call { return &MockDataCoord_GetInsertBinlogPaths_Call{Call: _e.mock.On("GetInsertBinlogPaths", _a0, _a1)} } @@ -1445,8 +1500,8 @@ type MockDataCoord_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 *MockDataCoord_Expecter) GetMetrics(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetMetrics_Call { return &MockDataCoord_GetMetrics_Call{Call: _e.mock.On("GetMetrics", _a0, _a1)} } @@ -1500,8 +1555,8 @@ type MockDataCoord_GetPartitionStatistics_Call struct { } // GetPartitionStatistics is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.GetPartitionStatisticsRequest +// - _a0 context.Context +// - _a1 *datapb.GetPartitionStatisticsRequest func (_e *MockDataCoord_Expecter) GetPartitionStatistics(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetPartitionStatistics_Call { return &MockDataCoord_GetPartitionStatistics_Call{Call: _e.mock.On("GetPartitionStatistics", _a0, _a1)} } @@ -1555,8 +1610,8 @@ type MockDataCoord_GetRecoveryInfo_Call struct { } // GetRecoveryInfo is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.GetRecoveryInfoRequest +// - _a0 context.Context +// - _a1 *datapb.GetRecoveryInfoRequest func (_e *MockDataCoord_Expecter) GetRecoveryInfo(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetRecoveryInfo_Call { return &MockDataCoord_GetRecoveryInfo_Call{Call: _e.mock.On("GetRecoveryInfo", _a0, _a1)} } @@ -1610,8 +1665,8 @@ type MockDataCoord_GetRecoveryInfoV2_Call struct { } // GetRecoveryInfoV2 is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.GetRecoveryInfoRequestV2 +// - _a0 context.Context +// - _a1 *datapb.GetRecoveryInfoRequestV2 func (_e *MockDataCoord_Expecter) GetRecoveryInfoV2(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetRecoveryInfoV2_Call { return &MockDataCoord_GetRecoveryInfoV2_Call{Call: _e.mock.On("GetRecoveryInfoV2", _a0, _a1)} } @@ -1665,8 +1720,8 @@ type MockDataCoord_GetSegmentIndexState_Call struct { } // GetSegmentIndexState is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *indexpb.GetSegmentIndexStateRequest +// - _a0 context.Context +// - _a1 *indexpb.GetSegmentIndexStateRequest func (_e *MockDataCoord_Expecter) GetSegmentIndexState(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetSegmentIndexState_Call { return &MockDataCoord_GetSegmentIndexState_Call{Call: _e.mock.On("GetSegmentIndexState", _a0, _a1)} } @@ -1720,8 +1775,8 @@ type MockDataCoord_GetSegmentInfo_Call struct { } // GetSegmentInfo is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.GetSegmentInfoRequest +// - _a0 context.Context +// - _a1 *datapb.GetSegmentInfoRequest func (_e *MockDataCoord_Expecter) GetSegmentInfo(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetSegmentInfo_Call { return &MockDataCoord_GetSegmentInfo_Call{Call: _e.mock.On("GetSegmentInfo", _a0, _a1)} } @@ -1775,8 +1830,8 @@ type MockDataCoord_GetSegmentInfoChannel_Call struct { } // GetSegmentInfoChannel is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.GetSegmentInfoChannelRequest +// - _a0 context.Context +// - _a1 *datapb.GetSegmentInfoChannelRequest func (_e *MockDataCoord_Expecter) GetSegmentInfoChannel(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetSegmentInfoChannel_Call { return &MockDataCoord_GetSegmentInfoChannel_Call{Call: _e.mock.On("GetSegmentInfoChannel", _a0, _a1)} } @@ -1830,8 +1885,8 @@ type MockDataCoord_GetSegmentStates_Call struct { } // GetSegmentStates is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.GetSegmentStatesRequest +// - _a0 context.Context +// - _a1 *datapb.GetSegmentStatesRequest func (_e *MockDataCoord_Expecter) GetSegmentStates(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetSegmentStates_Call { return &MockDataCoord_GetSegmentStates_Call{Call: _e.mock.On("GetSegmentStates", _a0, _a1)} } @@ -1885,8 +1940,8 @@ type MockDataCoord_GetSegmentsByStates_Call struct { } // GetSegmentsByStates is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.GetSegmentsByStatesRequest +// - _a0 context.Context +// - _a1 *datapb.GetSegmentsByStatesRequest func (_e *MockDataCoord_Expecter) GetSegmentsByStates(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetSegmentsByStates_Call { return &MockDataCoord_GetSegmentsByStates_Call{Call: _e.mock.On("GetSegmentsByStates", _a0, _a1)} } @@ -1940,8 +1995,8 @@ type MockDataCoord_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 *MockDataCoord_Expecter) GetStatisticsChannel(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetStatisticsChannel_Call { return &MockDataCoord_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", _a0, _a1)} } @@ -1995,8 +2050,8 @@ type MockDataCoord_GetTimeTickChannel_Call struct { } // GetTimeTickChannel is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *internalpb.GetTimeTickChannelRequest +// - _a0 context.Context +// - _a1 *internalpb.GetTimeTickChannelRequest func (_e *MockDataCoord_Expecter) GetTimeTickChannel(_a0 interface{}, _a1 interface{}) *MockDataCoord_GetTimeTickChannel_Call { return &MockDataCoord_GetTimeTickChannel_Call{Call: _e.mock.On("GetTimeTickChannel", _a0, _a1)} } @@ -2050,8 +2105,8 @@ type MockDataCoord_ImportV2_Call struct { } // ImportV2 is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *internalpb.ImportRequestInternal +// - _a0 context.Context +// - _a1 *internalpb.ImportRequestInternal func (_e *MockDataCoord_Expecter) ImportV2(_a0 interface{}, _a1 interface{}) *MockDataCoord_ImportV2_Call { return &MockDataCoord_ImportV2_Call{Call: _e.mock.On("ImportV2", _a0, _a1)} } @@ -2146,8 +2201,8 @@ type MockDataCoord_ListImports_Call struct { } // ListImports is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *internalpb.ListImportsRequestInternal +// - _a0 context.Context +// - _a1 *internalpb.ListImportsRequestInternal func (_e *MockDataCoord_Expecter) ListImports(_a0 interface{}, _a1 interface{}) *MockDataCoord_ListImports_Call { return &MockDataCoord_ListImports_Call{Call: _e.mock.On("ListImports", _a0, _a1)} } @@ -2201,8 +2256,8 @@ type MockDataCoord_ListIndexes_Call struct { } // ListIndexes is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *indexpb.ListIndexesRequest +// - _a0 context.Context +// - _a1 *indexpb.ListIndexesRequest func (_e *MockDataCoord_Expecter) ListIndexes(_a0 interface{}, _a1 interface{}) *MockDataCoord_ListIndexes_Call { return &MockDataCoord_ListIndexes_Call{Call: _e.mock.On("ListIndexes", _a0, _a1)} } @@ -2256,8 +2311,8 @@ type MockDataCoord_ManualCompaction_Call struct { } // ManualCompaction is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *milvuspb.ManualCompactionRequest +// - _a0 context.Context +// - _a1 *milvuspb.ManualCompactionRequest func (_e *MockDataCoord_Expecter) ManualCompaction(_a0 interface{}, _a1 interface{}) *MockDataCoord_ManualCompaction_Call { return &MockDataCoord_ManualCompaction_Call{Call: _e.mock.On("ManualCompaction", _a0, _a1)} } @@ -2311,8 +2366,8 @@ type MockDataCoord_MarkSegmentsDropped_Call struct { } // MarkSegmentsDropped is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.MarkSegmentsDroppedRequest +// - _a0 context.Context +// - _a1 *datapb.MarkSegmentsDroppedRequest func (_e *MockDataCoord_Expecter) MarkSegmentsDropped(_a0 interface{}, _a1 interface{}) *MockDataCoord_MarkSegmentsDropped_Call { return &MockDataCoord_MarkSegmentsDropped_Call{Call: _e.mock.On("MarkSegmentsDropped", _a0, _a1)} } @@ -2440,8 +2495,8 @@ type MockDataCoord_ReportDataNodeTtMsgs_Call struct { } // ReportDataNodeTtMsgs is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.ReportDataNodeTtMsgsRequest +// - _a0 context.Context +// - _a1 *datapb.ReportDataNodeTtMsgsRequest func (_e *MockDataCoord_Expecter) ReportDataNodeTtMsgs(_a0 interface{}, _a1 interface{}) *MockDataCoord_ReportDataNodeTtMsgs_Call { return &MockDataCoord_ReportDataNodeTtMsgs_Call{Call: _e.mock.On("ReportDataNodeTtMsgs", _a0, _a1)} } @@ -2495,8 +2550,8 @@ type MockDataCoord_SaveBinlogPaths_Call struct { } // SaveBinlogPaths is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.SaveBinlogPathsRequest +// - _a0 context.Context +// - _a1 *datapb.SaveBinlogPathsRequest func (_e *MockDataCoord_Expecter) SaveBinlogPaths(_a0 interface{}, _a1 interface{}) *MockDataCoord_SaveBinlogPaths_Call { return &MockDataCoord_SaveBinlogPaths_Call{Call: _e.mock.On("SaveBinlogPaths", _a0, _a1)} } @@ -2529,7 +2584,7 @@ type MockDataCoord_SetAddress_Call struct { } // SetAddress is a helper method to define mock.On call -// - address string +// - address string func (_e *MockDataCoord_Expecter) SetAddress(address interface{}) *MockDataCoord_SetAddress_Call { return &MockDataCoord_SetAddress_Call{Call: _e.mock.On("SetAddress", address)} } @@ -2562,7 +2617,7 @@ type MockDataCoord_SetDataNodeCreator_Call struct { } // SetDataNodeCreator is a helper method to define mock.On call -// - _a0 func(context.Context , string , int64)(types.DataNodeClient , error) +// - _a0 func(context.Context , string , int64)(types.DataNodeClient , error) func (_e *MockDataCoord_Expecter) SetDataNodeCreator(_a0 interface{}) *MockDataCoord_SetDataNodeCreator_Call { return &MockDataCoord_SetDataNodeCreator_Call{Call: _e.mock.On("SetDataNodeCreator", _a0)} } @@ -2595,7 +2650,7 @@ type MockDataCoord_SetEtcdClient_Call struct { } // SetEtcdClient is a helper method to define mock.On call -// - etcdClient *clientv3.Client +// - etcdClient *clientv3.Client func (_e *MockDataCoord_Expecter) SetEtcdClient(etcdClient interface{}) *MockDataCoord_SetEtcdClient_Call { return &MockDataCoord_SetEtcdClient_Call{Call: _e.mock.On("SetEtcdClient", etcdClient)} } @@ -2628,7 +2683,7 @@ type MockDataCoord_SetIndexNodeCreator_Call struct { } // SetIndexNodeCreator is a helper method to define mock.On call -// - _a0 func(context.Context , string , int64)(types.IndexNodeClient , error) +// - _a0 func(context.Context , string , int64)(types.IndexNodeClient , error) func (_e *MockDataCoord_Expecter) SetIndexNodeCreator(_a0 interface{}) *MockDataCoord_SetIndexNodeCreator_Call { return &MockDataCoord_SetIndexNodeCreator_Call{Call: _e.mock.On("SetIndexNodeCreator", _a0)} } @@ -2661,7 +2716,7 @@ type MockDataCoord_SetRootCoordClient_Call struct { } // SetRootCoordClient is a helper method to define mock.On call -// - rootCoord types.RootCoordClient +// - rootCoord types.RootCoordClient func (_e *MockDataCoord_Expecter) SetRootCoordClient(rootCoord interface{}) *MockDataCoord_SetRootCoordClient_Call { return &MockDataCoord_SetRootCoordClient_Call{Call: _e.mock.On("SetRootCoordClient", rootCoord)} } @@ -2715,8 +2770,8 @@ type MockDataCoord_SetSegmentState_Call struct { } // SetSegmentState is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.SetSegmentStateRequest +// - _a0 context.Context +// - _a1 *datapb.SetSegmentStateRequest func (_e *MockDataCoord_Expecter) SetSegmentState(_a0 interface{}, _a1 interface{}) *MockDataCoord_SetSegmentState_Call { return &MockDataCoord_SetSegmentState_Call{Call: _e.mock.On("SetSegmentState", _a0, _a1)} } @@ -2749,7 +2804,7 @@ type MockDataCoord_SetTiKVClient_Call struct { } // SetTiKVClient is a helper method to define mock.On call -// - client *txnkv.Client +// - client *txnkv.Client func (_e *MockDataCoord_Expecter) SetTiKVClient(client interface{}) *MockDataCoord_SetTiKVClient_Call { return &MockDataCoord_SetTiKVClient_Call{Call: _e.mock.On("SetTiKVClient", client)} } @@ -2803,8 +2858,8 @@ type MockDataCoord_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 *MockDataCoord_Expecter) ShowConfigurations(_a0 interface{}, _a1 interface{}) *MockDataCoord_ShowConfigurations_Call { return &MockDataCoord_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", _a0, _a1)} } @@ -2940,8 +2995,8 @@ type MockDataCoord_UpdateChannelCheckpoint_Call struct { } // UpdateChannelCheckpoint is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.UpdateChannelCheckpointRequest +// - _a0 context.Context +// - _a1 *datapb.UpdateChannelCheckpointRequest func (_e *MockDataCoord_Expecter) UpdateChannelCheckpoint(_a0 interface{}, _a1 interface{}) *MockDataCoord_UpdateChannelCheckpoint_Call { return &MockDataCoord_UpdateChannelCheckpoint_Call{Call: _e.mock.On("UpdateChannelCheckpoint", _a0, _a1)} } @@ -2995,8 +3050,8 @@ type MockDataCoord_UpdateSegmentStatistics_Call struct { } // UpdateSegmentStatistics is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.UpdateSegmentStatisticsRequest +// - _a0 context.Context +// - _a1 *datapb.UpdateSegmentStatisticsRequest func (_e *MockDataCoord_Expecter) UpdateSegmentStatistics(_a0 interface{}, _a1 interface{}) *MockDataCoord_UpdateSegmentStatistics_Call { return &MockDataCoord_UpdateSegmentStatistics_Call{Call: _e.mock.On("UpdateSegmentStatistics", _a0, _a1)} } @@ -3050,8 +3105,8 @@ type MockDataCoord_WatchChannels_Call struct { } // WatchChannels is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.WatchChannelsRequest +// - _a0 context.Context +// - _a1 *datapb.WatchChannelsRequest func (_e *MockDataCoord_Expecter) WatchChannels(_a0 interface{}, _a1 interface{}) *MockDataCoord_WatchChannels_Call { return &MockDataCoord_WatchChannels_Call{Call: _e.mock.On("WatchChannels", _a0, _a1)} } diff --git a/internal/mocks/mock_datacoord_client.go b/internal/mocks/mock_datacoord_client.go index 8f53648f30..67dc36fa16 100644 --- a/internal/mocks/mock_datacoord_client.go +++ b/internal/mocks/mock_datacoord_client.go @@ -142,9 +142,9 @@ type MockDataCoordClient_AlterIndex_Call struct { } // AlterIndex is a helper method to define mock.On call -// - ctx context.Context -// - in *indexpb.AlterIndexRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *indexpb.AlterIndexRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) AlterIndex(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_AlterIndex_Call { return &MockDataCoordClient_AlterIndex_Call{Call: _e.mock.On("AlterIndex", append([]interface{}{ctx, in}, opts...)...)} @@ -212,9 +212,9 @@ type MockDataCoordClient_AssignSegmentID_Call struct { } // AssignSegmentID is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.AssignSegmentIDRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.AssignSegmentIDRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) AssignSegmentID(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_AssignSegmentID_Call { return &MockDataCoordClient_AssignSegmentID_Call{Call: _e.mock.On("AssignSegmentID", append([]interface{}{ctx, in}, opts...)...)} @@ -282,9 +282,9 @@ type MockDataCoordClient_BroadcastAlteredCollection_Call struct { } // BroadcastAlteredCollection is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.AlterCollectionRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.AlterCollectionRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) BroadcastAlteredCollection(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_BroadcastAlteredCollection_Call { return &MockDataCoordClient_BroadcastAlteredCollection_Call{Call: _e.mock.On("BroadcastAlteredCollection", append([]interface{}{ctx, in}, opts...)...)} @@ -352,9 +352,9 @@ type MockDataCoordClient_CheckHealth_Call struct { } // CheckHealth is a helper method to define mock.On call -// - ctx context.Context -// - in *milvuspb.CheckHealthRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *milvuspb.CheckHealthRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) CheckHealth(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_CheckHealth_Call { return &MockDataCoordClient_CheckHealth_Call{Call: _e.mock.On("CheckHealth", append([]interface{}{ctx, in}, opts...)...)} @@ -463,9 +463,9 @@ type MockDataCoordClient_CreateIndex_Call struct { } // CreateIndex is a helper method to define mock.On call -// - ctx context.Context -// - in *indexpb.CreateIndexRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *indexpb.CreateIndexRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) CreateIndex(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_CreateIndex_Call { return &MockDataCoordClient_CreateIndex_Call{Call: _e.mock.On("CreateIndex", append([]interface{}{ctx, in}, opts...)...)} @@ -533,9 +533,9 @@ type MockDataCoordClient_DescribeIndex_Call struct { } // DescribeIndex is a helper method to define mock.On call -// - ctx context.Context -// - in *indexpb.DescribeIndexRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *indexpb.DescribeIndexRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) DescribeIndex(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_DescribeIndex_Call { return &MockDataCoordClient_DescribeIndex_Call{Call: _e.mock.On("DescribeIndex", append([]interface{}{ctx, in}, opts...)...)} @@ -603,9 +603,9 @@ type MockDataCoordClient_DropIndex_Call struct { } // DropIndex is a helper method to define mock.On call -// - ctx context.Context -// - in *indexpb.DropIndexRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *indexpb.DropIndexRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) DropIndex(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_DropIndex_Call { return &MockDataCoordClient_DropIndex_Call{Call: _e.mock.On("DropIndex", append([]interface{}{ctx, in}, opts...)...)} @@ -673,9 +673,9 @@ type MockDataCoordClient_DropVirtualChannel_Call struct { } // DropVirtualChannel is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.DropVirtualChannelRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.DropVirtualChannelRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) DropVirtualChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_DropVirtualChannel_Call { return &MockDataCoordClient_DropVirtualChannel_Call{Call: _e.mock.On("DropVirtualChannel", append([]interface{}{ctx, in}, opts...)...)} @@ -743,9 +743,9 @@ type MockDataCoordClient_Flush_Call struct { } // Flush is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.FlushRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.FlushRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) Flush(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_Flush_Call { return &MockDataCoordClient_Flush_Call{Call: _e.mock.On("Flush", append([]interface{}{ctx, in}, opts...)...)} @@ -813,9 +813,9 @@ type MockDataCoordClient_GcConfirm_Call struct { } // GcConfirm is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.GcConfirmRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.GcConfirmRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GcConfirm(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GcConfirm_Call { return &MockDataCoordClient_GcConfirm_Call{Call: _e.mock.On("GcConfirm", append([]interface{}{ctx, in}, opts...)...)} @@ -883,9 +883,9 @@ type MockDataCoordClient_GcControl_Call struct { } // GcControl is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.GcControlRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.GcControlRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GcControl(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GcControl_Call { return &MockDataCoordClient_GcControl_Call{Call: _e.mock.On("GcControl", append([]interface{}{ctx, in}, opts...)...)} @@ -914,6 +914,76 @@ func (_c *MockDataCoordClient_GcControl_Call) RunAndReturn(run func(context.Cont return _c } +// GetChannelRecoveryInfo provides a mock function with given fields: ctx, in, opts +func (_m *MockDataCoordClient) GetChannelRecoveryInfo(ctx context.Context, in *datapb.GetChannelRecoveryInfoRequest, opts ...grpc.CallOption) (*datapb.GetChannelRecoveryInfoResponse, error) { + _va := make([]interface{}, len(opts)) + for _i := range opts { + _va[_i] = opts[_i] + } + var _ca []interface{} + _ca = append(_ca, ctx, in) + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + var r0 *datapb.GetChannelRecoveryInfoResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *datapb.GetChannelRecoveryInfoRequest, ...grpc.CallOption) (*datapb.GetChannelRecoveryInfoResponse, error)); ok { + return rf(ctx, in, opts...) + } + if rf, ok := ret.Get(0).(func(context.Context, *datapb.GetChannelRecoveryInfoRequest, ...grpc.CallOption) *datapb.GetChannelRecoveryInfoResponse); ok { + r0 = rf(ctx, in, opts...) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*datapb.GetChannelRecoveryInfoResponse) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *datapb.GetChannelRecoveryInfoRequest, ...grpc.CallOption) error); ok { + r1 = rf(ctx, in, opts...) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockDataCoordClient_GetChannelRecoveryInfo_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetChannelRecoveryInfo' +type MockDataCoordClient_GetChannelRecoveryInfo_Call struct { + *mock.Call +} + +// GetChannelRecoveryInfo is a helper method to define mock.On call +// - ctx context.Context +// - in *datapb.GetChannelRecoveryInfoRequest +// - opts ...grpc.CallOption +func (_e *MockDataCoordClient_Expecter) GetChannelRecoveryInfo(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetChannelRecoveryInfo_Call { + return &MockDataCoordClient_GetChannelRecoveryInfo_Call{Call: _e.mock.On("GetChannelRecoveryInfo", + append([]interface{}{ctx, in}, opts...)...)} +} + +func (_c *MockDataCoordClient_GetChannelRecoveryInfo_Call) Run(run func(ctx context.Context, in *datapb.GetChannelRecoveryInfoRequest, opts ...grpc.CallOption)) *MockDataCoordClient_GetChannelRecoveryInfo_Call { + _c.Call.Run(func(args mock.Arguments) { + variadicArgs := make([]grpc.CallOption, len(args)-2) + for i, a := range args[2:] { + if a != nil { + variadicArgs[i] = a.(grpc.CallOption) + } + } + run(args[0].(context.Context), args[1].(*datapb.GetChannelRecoveryInfoRequest), variadicArgs...) + }) + return _c +} + +func (_c *MockDataCoordClient_GetChannelRecoveryInfo_Call) Return(_a0 *datapb.GetChannelRecoveryInfoResponse, _a1 error) *MockDataCoordClient_GetChannelRecoveryInfo_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockDataCoordClient_GetChannelRecoveryInfo_Call) RunAndReturn(run func(context.Context, *datapb.GetChannelRecoveryInfoRequest, ...grpc.CallOption) (*datapb.GetChannelRecoveryInfoResponse, error)) *MockDataCoordClient_GetChannelRecoveryInfo_Call { + _c.Call.Return(run) + return _c +} + // GetCollectionStatistics provides a mock function with given fields: ctx, in, opts func (_m *MockDataCoordClient) GetCollectionStatistics(ctx context.Context, in *datapb.GetCollectionStatisticsRequest, opts ...grpc.CallOption) (*datapb.GetCollectionStatisticsResponse, error) { _va := make([]interface{}, len(opts)) @@ -953,9 +1023,9 @@ type MockDataCoordClient_GetCollectionStatistics_Call struct { } // GetCollectionStatistics is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.GetCollectionStatisticsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.GetCollectionStatisticsRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetCollectionStatistics(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetCollectionStatistics_Call { return &MockDataCoordClient_GetCollectionStatistics_Call{Call: _e.mock.On("GetCollectionStatistics", append([]interface{}{ctx, in}, opts...)...)} @@ -1023,9 +1093,9 @@ type MockDataCoordClient_GetCompactionState_Call struct { } // GetCompactionState is a helper method to define mock.On call -// - ctx context.Context -// - in *milvuspb.GetCompactionStateRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *milvuspb.GetCompactionStateRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetCompactionState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetCompactionState_Call { return &MockDataCoordClient_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState", append([]interface{}{ctx, in}, opts...)...)} @@ -1093,9 +1163,9 @@ type MockDataCoordClient_GetCompactionStateWithPlans_Call struct { } // GetCompactionStateWithPlans is a helper method to define mock.On call -// - ctx context.Context -// - in *milvuspb.GetCompactionPlansRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *milvuspb.GetCompactionPlansRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetCompactionStateWithPlans(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetCompactionStateWithPlans_Call { return &MockDataCoordClient_GetCompactionStateWithPlans_Call{Call: _e.mock.On("GetCompactionStateWithPlans", append([]interface{}{ctx, in}, opts...)...)} @@ -1163,9 +1233,9 @@ type MockDataCoordClient_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 *MockDataCoordClient_Expecter) GetComponentStates(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetComponentStates_Call { return &MockDataCoordClient_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", append([]interface{}{ctx, in}, opts...)...)} @@ -1233,9 +1303,9 @@ type MockDataCoordClient_GetFlushAllState_Call struct { } // GetFlushAllState is a helper method to define mock.On call -// - ctx context.Context -// - in *milvuspb.GetFlushAllStateRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *milvuspb.GetFlushAllStateRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetFlushAllState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetFlushAllState_Call { return &MockDataCoordClient_GetFlushAllState_Call{Call: _e.mock.On("GetFlushAllState", append([]interface{}{ctx, in}, opts...)...)} @@ -1303,9 +1373,9 @@ type MockDataCoordClient_GetFlushState_Call struct { } // GetFlushState is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.GetFlushStateRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.GetFlushStateRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetFlushState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetFlushState_Call { return &MockDataCoordClient_GetFlushState_Call{Call: _e.mock.On("GetFlushState", append([]interface{}{ctx, in}, opts...)...)} @@ -1373,9 +1443,9 @@ type MockDataCoordClient_GetFlushedSegments_Call struct { } // GetFlushedSegments is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.GetFlushedSegmentsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.GetFlushedSegmentsRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetFlushedSegments(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetFlushedSegments_Call { return &MockDataCoordClient_GetFlushedSegments_Call{Call: _e.mock.On("GetFlushedSegments", append([]interface{}{ctx, in}, opts...)...)} @@ -1443,9 +1513,9 @@ type MockDataCoordClient_GetImportProgress_Call struct { } // GetImportProgress is a helper method to define mock.On call -// - ctx context.Context -// - in *internalpb.GetImportProgressRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *internalpb.GetImportProgressRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetImportProgress(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetImportProgress_Call { return &MockDataCoordClient_GetImportProgress_Call{Call: _e.mock.On("GetImportProgress", append([]interface{}{ctx, in}, opts...)...)} @@ -1513,9 +1583,9 @@ type MockDataCoordClient_GetIndexBuildProgress_Call struct { } // GetIndexBuildProgress is a helper method to define mock.On call -// - ctx context.Context -// - in *indexpb.GetIndexBuildProgressRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *indexpb.GetIndexBuildProgressRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetIndexBuildProgress(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetIndexBuildProgress_Call { return &MockDataCoordClient_GetIndexBuildProgress_Call{Call: _e.mock.On("GetIndexBuildProgress", append([]interface{}{ctx, in}, opts...)...)} @@ -1583,9 +1653,9 @@ type MockDataCoordClient_GetIndexInfos_Call struct { } // GetIndexInfos is a helper method to define mock.On call -// - ctx context.Context -// - in *indexpb.GetIndexInfoRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *indexpb.GetIndexInfoRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetIndexInfos(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetIndexInfos_Call { return &MockDataCoordClient_GetIndexInfos_Call{Call: _e.mock.On("GetIndexInfos", append([]interface{}{ctx, in}, opts...)...)} @@ -1653,9 +1723,9 @@ type MockDataCoordClient_GetIndexState_Call struct { } // GetIndexState is a helper method to define mock.On call -// - ctx context.Context -// - in *indexpb.GetIndexStateRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *indexpb.GetIndexStateRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetIndexState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetIndexState_Call { return &MockDataCoordClient_GetIndexState_Call{Call: _e.mock.On("GetIndexState", append([]interface{}{ctx, in}, opts...)...)} @@ -1723,9 +1793,9 @@ type MockDataCoordClient_GetIndexStatistics_Call struct { } // GetIndexStatistics is a helper method to define mock.On call -// - ctx context.Context -// - in *indexpb.GetIndexStatisticsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *indexpb.GetIndexStatisticsRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetIndexStatistics(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetIndexStatistics_Call { return &MockDataCoordClient_GetIndexStatistics_Call{Call: _e.mock.On("GetIndexStatistics", append([]interface{}{ctx, in}, opts...)...)} @@ -1793,9 +1863,9 @@ type MockDataCoordClient_GetInsertBinlogPaths_Call struct { } // GetInsertBinlogPaths is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.GetInsertBinlogPathsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.GetInsertBinlogPathsRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetInsertBinlogPaths(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetInsertBinlogPaths_Call { return &MockDataCoordClient_GetInsertBinlogPaths_Call{Call: _e.mock.On("GetInsertBinlogPaths", append([]interface{}{ctx, in}, opts...)...)} @@ -1863,9 +1933,9 @@ type MockDataCoordClient_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 *MockDataCoordClient_Expecter) GetMetrics(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetMetrics_Call { return &MockDataCoordClient_GetMetrics_Call{Call: _e.mock.On("GetMetrics", append([]interface{}{ctx, in}, opts...)...)} @@ -1933,9 +2003,9 @@ type MockDataCoordClient_GetPartitionStatistics_Call struct { } // GetPartitionStatistics is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.GetPartitionStatisticsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.GetPartitionStatisticsRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetPartitionStatistics(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetPartitionStatistics_Call { return &MockDataCoordClient_GetPartitionStatistics_Call{Call: _e.mock.On("GetPartitionStatistics", append([]interface{}{ctx, in}, opts...)...)} @@ -2003,9 +2073,9 @@ type MockDataCoordClient_GetRecoveryInfo_Call struct { } // GetRecoveryInfo is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.GetRecoveryInfoRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.GetRecoveryInfoRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetRecoveryInfo(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetRecoveryInfo_Call { return &MockDataCoordClient_GetRecoveryInfo_Call{Call: _e.mock.On("GetRecoveryInfo", append([]interface{}{ctx, in}, opts...)...)} @@ -2073,9 +2143,9 @@ type MockDataCoordClient_GetRecoveryInfoV2_Call struct { } // GetRecoveryInfoV2 is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.GetRecoveryInfoRequestV2 -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.GetRecoveryInfoRequestV2 +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetRecoveryInfoV2(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetRecoveryInfoV2_Call { return &MockDataCoordClient_GetRecoveryInfoV2_Call{Call: _e.mock.On("GetRecoveryInfoV2", append([]interface{}{ctx, in}, opts...)...)} @@ -2143,9 +2213,9 @@ type MockDataCoordClient_GetSegmentIndexState_Call struct { } // GetSegmentIndexState is a helper method to define mock.On call -// - ctx context.Context -// - in *indexpb.GetSegmentIndexStateRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *indexpb.GetSegmentIndexStateRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetSegmentIndexState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetSegmentIndexState_Call { return &MockDataCoordClient_GetSegmentIndexState_Call{Call: _e.mock.On("GetSegmentIndexState", append([]interface{}{ctx, in}, opts...)...)} @@ -2213,9 +2283,9 @@ type MockDataCoordClient_GetSegmentInfo_Call struct { } // GetSegmentInfo is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.GetSegmentInfoRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.GetSegmentInfoRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetSegmentInfo(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetSegmentInfo_Call { return &MockDataCoordClient_GetSegmentInfo_Call{Call: _e.mock.On("GetSegmentInfo", append([]interface{}{ctx, in}, opts...)...)} @@ -2283,9 +2353,9 @@ type MockDataCoordClient_GetSegmentInfoChannel_Call struct { } // GetSegmentInfoChannel is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.GetSegmentInfoChannelRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.GetSegmentInfoChannelRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetSegmentInfoChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetSegmentInfoChannel_Call { return &MockDataCoordClient_GetSegmentInfoChannel_Call{Call: _e.mock.On("GetSegmentInfoChannel", append([]interface{}{ctx, in}, opts...)...)} @@ -2353,9 +2423,9 @@ type MockDataCoordClient_GetSegmentStates_Call struct { } // GetSegmentStates is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.GetSegmentStatesRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.GetSegmentStatesRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetSegmentStates(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetSegmentStates_Call { return &MockDataCoordClient_GetSegmentStates_Call{Call: _e.mock.On("GetSegmentStates", append([]interface{}{ctx, in}, opts...)...)} @@ -2423,9 +2493,9 @@ type MockDataCoordClient_GetSegmentsByStates_Call struct { } // GetSegmentsByStates is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.GetSegmentsByStatesRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.GetSegmentsByStatesRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetSegmentsByStates(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetSegmentsByStates_Call { return &MockDataCoordClient_GetSegmentsByStates_Call{Call: _e.mock.On("GetSegmentsByStates", append([]interface{}{ctx, in}, opts...)...)} @@ -2493,9 +2563,9 @@ type MockDataCoordClient_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 *MockDataCoordClient_Expecter) GetStatisticsChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetStatisticsChannel_Call { return &MockDataCoordClient_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", append([]interface{}{ctx, in}, opts...)...)} @@ -2563,9 +2633,9 @@ type MockDataCoordClient_GetTimeTickChannel_Call struct { } // GetTimeTickChannel is a helper method to define mock.On call -// - ctx context.Context -// - in *internalpb.GetTimeTickChannelRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *internalpb.GetTimeTickChannelRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) GetTimeTickChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_GetTimeTickChannel_Call { return &MockDataCoordClient_GetTimeTickChannel_Call{Call: _e.mock.On("GetTimeTickChannel", append([]interface{}{ctx, in}, opts...)...)} @@ -2633,9 +2703,9 @@ type MockDataCoordClient_ImportV2_Call struct { } // ImportV2 is a helper method to define mock.On call -// - ctx context.Context -// - in *internalpb.ImportRequestInternal -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *internalpb.ImportRequestInternal +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) ImportV2(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_ImportV2_Call { return &MockDataCoordClient_ImportV2_Call{Call: _e.mock.On("ImportV2", append([]interface{}{ctx, in}, opts...)...)} @@ -2703,9 +2773,9 @@ type MockDataCoordClient_ListImports_Call struct { } // ListImports is a helper method to define mock.On call -// - ctx context.Context -// - in *internalpb.ListImportsRequestInternal -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *internalpb.ListImportsRequestInternal +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) ListImports(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_ListImports_Call { return &MockDataCoordClient_ListImports_Call{Call: _e.mock.On("ListImports", append([]interface{}{ctx, in}, opts...)...)} @@ -2773,9 +2843,9 @@ type MockDataCoordClient_ListIndexes_Call struct { } // ListIndexes is a helper method to define mock.On call -// - ctx context.Context -// - in *indexpb.ListIndexesRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *indexpb.ListIndexesRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) ListIndexes(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_ListIndexes_Call { return &MockDataCoordClient_ListIndexes_Call{Call: _e.mock.On("ListIndexes", append([]interface{}{ctx, in}, opts...)...)} @@ -2843,9 +2913,9 @@ type MockDataCoordClient_ManualCompaction_Call struct { } // ManualCompaction is a helper method to define mock.On call -// - ctx context.Context -// - in *milvuspb.ManualCompactionRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *milvuspb.ManualCompactionRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) ManualCompaction(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_ManualCompaction_Call { return &MockDataCoordClient_ManualCompaction_Call{Call: _e.mock.On("ManualCompaction", append([]interface{}{ctx, in}, opts...)...)} @@ -2913,9 +2983,9 @@ type MockDataCoordClient_MarkSegmentsDropped_Call struct { } // MarkSegmentsDropped is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.MarkSegmentsDroppedRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.MarkSegmentsDroppedRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) MarkSegmentsDropped(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_MarkSegmentsDropped_Call { return &MockDataCoordClient_MarkSegmentsDropped_Call{Call: _e.mock.On("MarkSegmentsDropped", append([]interface{}{ctx, in}, opts...)...)} @@ -2983,9 +3053,9 @@ type MockDataCoordClient_ReportDataNodeTtMsgs_Call struct { } // ReportDataNodeTtMsgs is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.ReportDataNodeTtMsgsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.ReportDataNodeTtMsgsRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) ReportDataNodeTtMsgs(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_ReportDataNodeTtMsgs_Call { return &MockDataCoordClient_ReportDataNodeTtMsgs_Call{Call: _e.mock.On("ReportDataNodeTtMsgs", append([]interface{}{ctx, in}, opts...)...)} @@ -3053,9 +3123,9 @@ type MockDataCoordClient_SaveBinlogPaths_Call struct { } // SaveBinlogPaths is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.SaveBinlogPathsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.SaveBinlogPathsRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) SaveBinlogPaths(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_SaveBinlogPaths_Call { return &MockDataCoordClient_SaveBinlogPaths_Call{Call: _e.mock.On("SaveBinlogPaths", append([]interface{}{ctx, in}, opts...)...)} @@ -3123,9 +3193,9 @@ type MockDataCoordClient_SetSegmentState_Call struct { } // SetSegmentState is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.SetSegmentStateRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.SetSegmentStateRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) SetSegmentState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_SetSegmentState_Call { return &MockDataCoordClient_SetSegmentState_Call{Call: _e.mock.On("SetSegmentState", append([]interface{}{ctx, in}, opts...)...)} @@ -3193,9 +3263,9 @@ type MockDataCoordClient_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 *MockDataCoordClient_Expecter) ShowConfigurations(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_ShowConfigurations_Call { return &MockDataCoordClient_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", append([]interface{}{ctx, in}, opts...)...)} @@ -3263,9 +3333,9 @@ type MockDataCoordClient_UpdateChannelCheckpoint_Call struct { } // UpdateChannelCheckpoint is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.UpdateChannelCheckpointRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.UpdateChannelCheckpointRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) UpdateChannelCheckpoint(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_UpdateChannelCheckpoint_Call { return &MockDataCoordClient_UpdateChannelCheckpoint_Call{Call: _e.mock.On("UpdateChannelCheckpoint", append([]interface{}{ctx, in}, opts...)...)} @@ -3333,9 +3403,9 @@ type MockDataCoordClient_UpdateSegmentStatistics_Call struct { } // UpdateSegmentStatistics is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.UpdateSegmentStatisticsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.UpdateSegmentStatisticsRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) UpdateSegmentStatistics(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_UpdateSegmentStatistics_Call { return &MockDataCoordClient_UpdateSegmentStatistics_Call{Call: _e.mock.On("UpdateSegmentStatistics", append([]interface{}{ctx, in}, opts...)...)} @@ -3403,9 +3473,9 @@ type MockDataCoordClient_WatchChannels_Call struct { } // WatchChannels is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.WatchChannelsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.WatchChannelsRequest +// - opts ...grpc.CallOption func (_e *MockDataCoordClient_Expecter) WatchChannels(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_WatchChannels_Call { return &MockDataCoordClient_WatchChannels_Call{Call: _e.mock.On("WatchChannels", append([]interface{}{ctx, in}, opts...)...)} diff --git a/internal/mocks/streamingnode/server/mock_flusher/mock_Flusher.go b/internal/mocks/streamingnode/server/mock_flusher/mock_Flusher.go new file mode 100644 index 0000000000..d4d8f524a0 --- /dev/null +++ b/internal/mocks/streamingnode/server/mock_flusher/mock_Flusher.go @@ -0,0 +1,242 @@ +// Code generated by mockery v2.32.4. DO NOT EDIT. + +package mock_flusher + +import ( + wal "github.com/milvus-io/milvus/internal/streamingnode/server/wal" + mock "github.com/stretchr/testify/mock" +) + +// MockFlusher is an autogenerated mock type for the Flusher type +type MockFlusher struct { + mock.Mock +} + +type MockFlusher_Expecter struct { + mock *mock.Mock +} + +func (_m *MockFlusher) EXPECT() *MockFlusher_Expecter { + return &MockFlusher_Expecter{mock: &_m.Mock} +} + +// RegisterPChannel provides a mock function with given fields: pchannel, w +func (_m *MockFlusher) RegisterPChannel(pchannel string, w wal.WAL) error { + ret := _m.Called(pchannel, w) + + var r0 error + if rf, ok := ret.Get(0).(func(string, wal.WAL) error); ok { + r0 = rf(pchannel, w) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockFlusher_RegisterPChannel_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RegisterPChannel' +type MockFlusher_RegisterPChannel_Call struct { + *mock.Call +} + +// RegisterPChannel is a helper method to define mock.On call +// - pchannel string +// - w wal.WAL +func (_e *MockFlusher_Expecter) RegisterPChannel(pchannel interface{}, w interface{}) *MockFlusher_RegisterPChannel_Call { + return &MockFlusher_RegisterPChannel_Call{Call: _e.mock.On("RegisterPChannel", pchannel, w)} +} + +func (_c *MockFlusher_RegisterPChannel_Call) Run(run func(pchannel string, w wal.WAL)) *MockFlusher_RegisterPChannel_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(string), args[1].(wal.WAL)) + }) + return _c +} + +func (_c *MockFlusher_RegisterPChannel_Call) Return(_a0 error) *MockFlusher_RegisterPChannel_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockFlusher_RegisterPChannel_Call) RunAndReturn(run func(string, wal.WAL) error) *MockFlusher_RegisterPChannel_Call { + _c.Call.Return(run) + return _c +} + +// RegisterVChannel provides a mock function with given fields: vchannel, _a1 +func (_m *MockFlusher) RegisterVChannel(vchannel string, _a1 wal.WAL) { + _m.Called(vchannel, _a1) +} + +// MockFlusher_RegisterVChannel_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RegisterVChannel' +type MockFlusher_RegisterVChannel_Call struct { + *mock.Call +} + +// RegisterVChannel is a helper method to define mock.On call +// - vchannel string +// - _a1 wal.WAL +func (_e *MockFlusher_Expecter) RegisterVChannel(vchannel interface{}, _a1 interface{}) *MockFlusher_RegisterVChannel_Call { + return &MockFlusher_RegisterVChannel_Call{Call: _e.mock.On("RegisterVChannel", vchannel, _a1)} +} + +func (_c *MockFlusher_RegisterVChannel_Call) Run(run func(vchannel string, _a1 wal.WAL)) *MockFlusher_RegisterVChannel_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(string), args[1].(wal.WAL)) + }) + return _c +} + +func (_c *MockFlusher_RegisterVChannel_Call) Return() *MockFlusher_RegisterVChannel_Call { + _c.Call.Return() + return _c +} + +func (_c *MockFlusher_RegisterVChannel_Call) RunAndReturn(run func(string, wal.WAL)) *MockFlusher_RegisterVChannel_Call { + _c.Call.Return(run) + return _c +} + +// Start provides a mock function with given fields: +func (_m *MockFlusher) Start() { + _m.Called() +} + +// MockFlusher_Start_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Start' +type MockFlusher_Start_Call struct { + *mock.Call +} + +// Start is a helper method to define mock.On call +func (_e *MockFlusher_Expecter) Start() *MockFlusher_Start_Call { + return &MockFlusher_Start_Call{Call: _e.mock.On("Start")} +} + +func (_c *MockFlusher_Start_Call) Run(run func()) *MockFlusher_Start_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockFlusher_Start_Call) Return() *MockFlusher_Start_Call { + _c.Call.Return() + return _c +} + +func (_c *MockFlusher_Start_Call) RunAndReturn(run func()) *MockFlusher_Start_Call { + _c.Call.Return(run) + return _c +} + +// Stop provides a mock function with given fields: +func (_m *MockFlusher) Stop() { + _m.Called() +} + +// MockFlusher_Stop_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Stop' +type MockFlusher_Stop_Call struct { + *mock.Call +} + +// Stop is a helper method to define mock.On call +func (_e *MockFlusher_Expecter) Stop() *MockFlusher_Stop_Call { + return &MockFlusher_Stop_Call{Call: _e.mock.On("Stop")} +} + +func (_c *MockFlusher_Stop_Call) Run(run func()) *MockFlusher_Stop_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockFlusher_Stop_Call) Return() *MockFlusher_Stop_Call { + _c.Call.Return() + return _c +} + +func (_c *MockFlusher_Stop_Call) RunAndReturn(run func()) *MockFlusher_Stop_Call { + _c.Call.Return(run) + return _c +} + +// UnregisterPChannel provides a mock function with given fields: pchannel +func (_m *MockFlusher) UnregisterPChannel(pchannel string) { + _m.Called(pchannel) +} + +// MockFlusher_UnregisterPChannel_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UnregisterPChannel' +type MockFlusher_UnregisterPChannel_Call struct { + *mock.Call +} + +// UnregisterPChannel is a helper method to define mock.On call +// - pchannel string +func (_e *MockFlusher_Expecter) UnregisterPChannel(pchannel interface{}) *MockFlusher_UnregisterPChannel_Call { + return &MockFlusher_UnregisterPChannel_Call{Call: _e.mock.On("UnregisterPChannel", pchannel)} +} + +func (_c *MockFlusher_UnregisterPChannel_Call) Run(run func(pchannel string)) *MockFlusher_UnregisterPChannel_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(string)) + }) + return _c +} + +func (_c *MockFlusher_UnregisterPChannel_Call) Return() *MockFlusher_UnregisterPChannel_Call { + _c.Call.Return() + return _c +} + +func (_c *MockFlusher_UnregisterPChannel_Call) RunAndReturn(run func(string)) *MockFlusher_UnregisterPChannel_Call { + _c.Call.Return(run) + return _c +} + +// UnregisterVChannel provides a mock function with given fields: vchannel +func (_m *MockFlusher) UnregisterVChannel(vchannel string) { + _m.Called(vchannel) +} + +// MockFlusher_UnregisterVChannel_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UnregisterVChannel' +type MockFlusher_UnregisterVChannel_Call struct { + *mock.Call +} + +// UnregisterVChannel is a helper method to define mock.On call +// - vchannel string +func (_e *MockFlusher_Expecter) UnregisterVChannel(vchannel interface{}) *MockFlusher_UnregisterVChannel_Call { + return &MockFlusher_UnregisterVChannel_Call{Call: _e.mock.On("UnregisterVChannel", vchannel)} +} + +func (_c *MockFlusher_UnregisterVChannel_Call) Run(run func(vchannel string)) *MockFlusher_UnregisterVChannel_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(string)) + }) + return _c +} + +func (_c *MockFlusher_UnregisterVChannel_Call) Return() *MockFlusher_UnregisterVChannel_Call { + _c.Call.Return() + return _c +} + +func (_c *MockFlusher_UnregisterVChannel_Call) RunAndReturn(run func(string)) *MockFlusher_UnregisterVChannel_Call { + _c.Call.Return(run) + return _c +} + +// NewMockFlusher creates a new instance of MockFlusher. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewMockFlusher(t interface { + mock.TestingT + Cleanup(func()) +}) *MockFlusher { + mock := &MockFlusher{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/internal/proto/data_coord.proto b/internal/proto/data_coord.proto index 3cd3fa4cd7..e329612812 100644 --- a/internal/proto/data_coord.proto +++ b/internal/proto/data_coord.proto @@ -54,6 +54,7 @@ service DataCoord { rpc SaveBinlogPaths(SaveBinlogPathsRequest) returns (common.Status){} rpc GetRecoveryInfo(GetRecoveryInfoRequest) returns (GetRecoveryInfoResponse){} rpc GetRecoveryInfoV2(GetRecoveryInfoRequestV2) returns (GetRecoveryInfoResponseV2){} + rpc GetChannelRecoveryInfo(GetChannelRecoveryInfoRequest) returns (GetChannelRecoveryInfoResponse){} rpc GetFlushedSegments(GetFlushedSegmentsRequest) returns(GetFlushedSegmentsResponse){} rpc GetSegmentsByStates(GetSegmentsByStatesRequest) returns(GetSegmentsByStatesResponse){} rpc GetFlushAllState(milvus.GetFlushAllStateRequest) returns(milvus.GetFlushAllStateResponse) {} @@ -463,6 +464,17 @@ message GetRecoveryInfoRequestV2 { repeated int64 partitionIDs = 3; } +message GetChannelRecoveryInfoRequest { + common.MsgBase base = 1; + string vchannel = 2; +} + +message GetChannelRecoveryInfoResponse { + common.Status status = 1; + VchannelInfo info = 2; + schema.CollectionSchema schema = 3; +} + message GetSegmentsByStatesRequest { common.MsgBase base = 1; int64 collectionID = 2; diff --git a/internal/streamingnode/server/flusher/flusher.go b/internal/streamingnode/server/flusher/flusher.go new file mode 100644 index 0000000000..594d5b3bc2 --- /dev/null +++ b/internal/streamingnode/server/flusher/flusher.go @@ -0,0 +1,40 @@ +// 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 flusher + +import "github.com/milvus-io/milvus/internal/streamingnode/server/wal" + +type Flusher interface { + // RegisterPChannel ASYNCHRONOUSLY creates and starts pipelines belonging to the pchannel/WAL. + // If a pipeline creation fails, the flusher will keep retrying to create it indefinitely. + RegisterPChannel(pchannel string, w wal.WAL) error + + // UnregisterPChannel stops and removes pipelines belonging to the pchannel. + UnregisterPChannel(pchannel string) + + // RegisterVChannel ASYNCHRONOUSLY create pipeline belonging to the vchannel. + RegisterVChannel(vchannel string, wal wal.WAL) + + // UnregisterVChannel stops and removes pipeline belonging to the vchannel. + UnregisterVChannel(vchannel string) + + // Start flusher service. + Start() + + // Stop flusher, will synchronously flush all remaining data. + Stop() +} diff --git a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go new file mode 100644 index 0000000000..e7b7b99fbe --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go @@ -0,0 +1,206 @@ +// 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 flusherimpl + +import ( + "context" + "sync" + "time" + + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/flushcommon/pipeline" + "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" + "github.com/milvus-io/milvus/internal/flushcommon/util" + "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + "github.com/milvus-io/milvus/internal/streamingnode/server/flusher" + "github.com/milvus-io/milvus/internal/streamingnode/server/resource" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal" + adaptor2 "github.com/milvus-io/milvus/internal/streamingnode/server/wal/adaptor" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/message/adaptor" + "github.com/milvus-io/milvus/pkg/streaming/util/options" + "github.com/milvus-io/milvus/pkg/util/funcutil" + "github.com/milvus-io/milvus/pkg/util/merr" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +var tickDuration = 3 * time.Second + +var _ flusher.Flusher = (*flusherImpl)(nil) + +type flusherImpl struct { + fgMgr pipeline.FlowgraphManager + syncMgr syncmgr.SyncManager + wbMgr writebuffer.BufferManager + cpUpdater *util.ChannelCheckpointUpdater + + tasks *typeutil.ConcurrentMap[string, wal.WAL] // unwatched vchannels + scanners *typeutil.ConcurrentMap[string, wal.Scanner] // watched scanners + + stopOnce sync.Once + stopChan chan struct{} +} + +func NewFlusher() flusher.Flusher { + params := GetPipelineParams() + fgMgr := pipeline.NewFlowgraphManager() + return &flusherImpl{ + fgMgr: fgMgr, + syncMgr: params.SyncMgr, + wbMgr: params.WriteBufferManager, + cpUpdater: params.CheckpointUpdater, + tasks: typeutil.NewConcurrentMap[string, wal.WAL](), + scanners: typeutil.NewConcurrentMap[string, wal.Scanner](), + stopOnce: sync.Once{}, + stopChan: make(chan struct{}), + } +} + +func (f *flusherImpl) RegisterPChannel(pchannel string, wal wal.WAL) error { + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + resp, err := resource.Resource().RootCoordClient().GetPChannelInfo(ctx, &rootcoordpb.GetPChannelInfoRequest{ + Pchannel: pchannel, + }) + if err = merr.CheckRPCCall(resp, err); err != nil { + return err + } + for _, collectionInfo := range resp.GetCollections() { + f.tasks.Insert(collectionInfo.GetVchannel(), wal) + } + return nil +} + +func (f *flusherImpl) UnregisterPChannel(pchannel string) { + f.scanners.Range(func(vchannel string, scanner wal.Scanner) bool { + if funcutil.ToPhysicalChannel(vchannel) != pchannel { + return true + } + f.UnregisterVChannel(vchannel) + return true + }) +} + +func (f *flusherImpl) RegisterVChannel(vchannel string, wal wal.WAL) { + f.tasks.Insert(vchannel, wal) +} + +func (f *flusherImpl) UnregisterVChannel(vchannel string) { + if scanner, ok := f.scanners.GetAndRemove(vchannel); ok { + err := scanner.Close() + if err != nil { + log.Warn("scanner error", zap.String("vchannel", vchannel), zap.Error(err)) + } + } + f.fgMgr.RemoveFlowgraph(vchannel) + f.wbMgr.RemoveChannel(vchannel) +} + +func (f *flusherImpl) Start() { + f.wbMgr.Start() + go f.cpUpdater.Start() + go func() { + ticker := time.NewTicker(tickDuration) + defer ticker.Stop() + for { + select { + case <-f.stopChan: + log.Info("flusher stopped") + return + case <-ticker.C: + f.tasks.Range(func(vchannel string, wal wal.WAL) bool { + err := f.buildPipeline(vchannel, wal) + if err != nil { + log.Warn("build pipeline failed", zap.String("vchannel", vchannel), zap.Error(err)) + return true + } + log.Info("build pipeline done", zap.String("vchannel", vchannel)) + f.tasks.Remove(vchannel) + return true + }) + } + } + }() +} + +func (f *flusherImpl) Stop() { + f.stopOnce.Do(func() { + close(f.stopChan) + f.scanners.Range(func(vchannel string, scanner wal.Scanner) bool { + err := scanner.Close() + if err != nil { + log.Warn("scanner error", zap.String("vchannel", vchannel), zap.Error(err)) + } + return true + }) + f.fgMgr.ClearFlowgraphs() + f.wbMgr.Stop() + f.cpUpdater.Close() + }) +} + +func (f *flusherImpl) buildPipeline(vchannel string, w wal.WAL) error { + if f.fgMgr.HasFlowgraph(vchannel) { + return nil + } + log.Info("start to build pipeline", zap.String("vchannel", vchannel)) + + // Get recovery info from datacoord. + ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) + defer cancel() + resp, err := resource.Resource().DataCoordClient(). + GetChannelRecoveryInfo(ctx, &datapb.GetChannelRecoveryInfoRequest{Vchannel: vchannel}) + if err = merr.CheckRPCCall(resp, err); err != nil { + return err + } + + // Convert common.MessageID to message.messageID. + mqWrapperID, err := adaptor.DeserializeToMQWrapperID(resp.GetInfo().GetSeekPosition().GetMsgID(), w.WALName()) + if err != nil { + return err + } + messageID := adaptor.MustGetMessageIDFromMQWrapperID(mqWrapperID) + + // Create scanner. + policy := options.DeliverPolicyStartFrom(messageID) + filter := func(msg message.ImmutableMessage) bool { return msg.VChannel() == vchannel } + handler := adaptor2.NewMsgPackAdaptorHandler() + ro := wal.ReadOption{ + DeliverPolicy: policy, + MessageFilter: filter, + MesasgeHandler: handler, + } + scanner, err := w.Read(ctx, ro) + if err != nil { + return err + } + + // Build and add pipeline. + ds, err := pipeline.NewStreamingNodeDataSyncService(ctx, GetPipelineParams(), + &datapb.ChannelWatchInfo{Vchan: resp.GetInfo(), Schema: resp.GetSchema()}, handler.Chan()) + if err != nil { + return err + } + ds.Start() + f.fgMgr.AddFlowgraph(ds) + f.scanners.Insert(vchannel, scanner) + return nil +} diff --git a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go new file mode 100644 index 0000000000..54170315cb --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go @@ -0,0 +1,185 @@ +// 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 flusherimpl + +import ( + "context" + "sync" + "testing" + "time" + + "github.com/samber/lo" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/suite" + "google.golang.org/grpc" + + "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/flushcommon/syncmgr" + "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" + "github.com/milvus-io/milvus/internal/mocks" + "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_wal" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + "github.com/milvus-io/milvus/internal/streamingnode/server/flusher" + "github.com/milvus-io/milvus/internal/streamingnode/server/resource" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal" + "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/util/paramtable" +) + +type FlusherSuite struct { + suite.Suite + + pchannel string + vchannels []string + + wbMgr *writebuffer.MockBufferManager + rootcoord *mocks.MockRootCoordClient + + wal wal.WAL + flusher flusher.Flusher +} + +func (s *FlusherSuite) SetupSuite() { + paramtable.Init() + tickDuration = 10 * time.Millisecond + + s.pchannel = "by-dev-rootcoord-dml_0" + s.vchannels = []string{ + "by-dev-rootcoord-dml_0_123456v0", + "by-dev-rootcoord-dml_0_123456v1", + "by-dev-rootcoord-dml_0_123456v2", + } + + rootcoord := mocks.NewMockRootCoordClient(s.T()) + + datacoord := mocks.NewMockDataCoordClient(s.T()) + datacoord.EXPECT().GetChannelRecoveryInfo(mock.Anything, mock.Anything).RunAndReturn( + func(ctx context.Context, request *datapb.GetChannelRecoveryInfoRequest, option ...grpc.CallOption, + ) (*datapb.GetChannelRecoveryInfoResponse, error) { + messageID := 1 + b := make([]byte, 8) + common.Endian.PutUint64(b, uint64(messageID)) + return &datapb.GetChannelRecoveryInfoResponse{ + Info: &datapb.VchannelInfo{ + ChannelName: request.GetVchannel(), + SeekPosition: &msgpb.MsgPosition{MsgID: b}, + }, + Schema: &schemapb.CollectionSchema{ + Fields: []*schemapb.FieldSchema{ + {FieldID: 100, Name: "ID", IsPrimaryKey: true}, + {FieldID: 101, Name: "Vector"}, + }, + }, + }, nil + }) + + syncMgr := syncmgr.NewMockSyncManager(s.T()) + wbMgr := writebuffer.NewMockBufferManager(s.T()) + wbMgr.EXPECT().Register(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) + wbMgr.EXPECT().RemoveChannel(mock.Anything).Return() + wbMgr.EXPECT().Start().Return() + wbMgr.EXPECT().Stop().Return() + + resource.Init( + resource.OptSyncManager(syncMgr), + resource.OptBufferManager(wbMgr), + resource.OptRootCoordClient(rootcoord), + resource.OptDataCoordClient(datacoord), + ) + + s.wbMgr = wbMgr + s.rootcoord = rootcoord +} + +func (s *FlusherSuite) SetupTest() { + handlers := make([]wal.MessageHandler, 0, len(s.vchannels)) + scanner := mock_wal.NewMockScanner(s.T()) + + w := mock_wal.NewMockWAL(s.T()) + w.EXPECT().WALName().Return("rocksmq") + w.EXPECT().Read(mock.Anything, mock.Anything).RunAndReturn( + func(ctx context.Context, option wal.ReadOption) (wal.Scanner, error) { + handlers = append(handlers, option.MesasgeHandler) + return scanner, nil + }) + + once := sync.Once{} + scanner.EXPECT().Close().RunAndReturn(func() error { + once.Do(func() { + for _, handler := range handlers { + handler.Close() + } + }) + return nil + }) + + s.wal = w + s.flusher = NewFlusher() + s.flusher.Start() +} + +func (s *FlusherSuite) TearDownTest() { + s.flusher.Stop() +} + +func (s *FlusherSuite) TestFlusher_RegisterPChannel() { + collectionsInfo := lo.Map(s.vchannels, func(vchannel string, i int) *rootcoordpb.CollectionInfoOnPChannel { + return &rootcoordpb.CollectionInfoOnPChannel{ + CollectionId: int64(i), + Partitions: []*rootcoordpb.PartitionInfoOnPChannel{{PartitionId: int64(i)}}, + Vchannel: vchannel, + } + }) + s.rootcoord.EXPECT().GetPChannelInfo(mock.Anything, mock.Anything). + Return(&rootcoordpb.GetPChannelInfoResponse{Collections: collectionsInfo}, nil) + + err := s.flusher.RegisterPChannel(s.pchannel, s.wal) + s.NoError(err) + + s.Eventually(func() bool { + return lo.EveryBy(s.vchannels, func(vchannel string) bool { + return s.flusher.(*flusherImpl).fgMgr.HasFlowgraph(vchannel) + }) + }, 10*time.Second, 10*time.Millisecond) + + s.flusher.UnregisterPChannel(s.pchannel) + s.Equal(0, s.flusher.(*flusherImpl).fgMgr.GetFlowgraphCount()) + s.Equal(0, s.flusher.(*flusherImpl).scanners.Len()) +} + +func (s *FlusherSuite) TestFlusher_RegisterVChannel() { + for _, vchannel := range s.vchannels { + s.flusher.RegisterVChannel(vchannel, s.wal) + } + s.Eventually(func() bool { + return lo.EveryBy(s.vchannels, func(vchannel string) bool { + return s.flusher.(*flusherImpl).fgMgr.HasFlowgraph(vchannel) + }) + }, 10*time.Second, 10*time.Millisecond) + + for _, vchannel := range s.vchannels { + s.flusher.UnregisterVChannel(vchannel) + } + s.Equal(0, s.flusher.(*flusherImpl).fgMgr.GetFlowgraphCount()) + s.Equal(0, s.flusher.(*flusherImpl).scanners.Len()) +} + +func TestFlusherSuite(t *testing.T) { + suite.Run(t, new(FlusherSuite)) +} diff --git a/internal/streamingnode/server/flusher/flusherimpl/flushmsg_handler_impl.go b/internal/streamingnode/server/flusher/flusherimpl/flushmsg_handler_impl.go new file mode 100644 index 0000000000..ae74cacd13 --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/flushmsg_handler_impl.go @@ -0,0 +1,36 @@ +// 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 flusherimpl + +import ( + "context" + + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" + "github.com/milvus-io/milvus/pkg/log" +) + +// TODO: func(vchannel string, msg FlushMsg) +func flushMsgHandlerImpl(wbMgr writebuffer.BufferManager) func(vchannel string, segmentIDs []int64) { + return func(vchannel string, segmentIDs []int64) { + err := wbMgr.SealSegments(context.Background(), vchannel, segmentIDs) + if err != nil { + log.Warn("failed to seal segments", zap.Error(err)) + } + } +} diff --git a/internal/streamingnode/server/flusher/flusherimpl/flushmsg_handler_impl_test.go b/internal/streamingnode/server/flusher/flusherimpl/flushmsg_handler_impl_test.go new file mode 100644 index 0000000000..cce193ad28 --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/flushmsg_handler_impl_test.go @@ -0,0 +1,42 @@ +// 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 flusherimpl + +import ( + "testing" + + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/mock" + + "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" +) + +func TestFlushMsgHandler(t *testing.T) { + // test failed + wbMgr := writebuffer.NewMockBufferManager(t) + wbMgr.EXPECT().SealSegments(mock.Anything, mock.Anything, mock.Anything).Return(errors.New("mock err")) + + fn := flushMsgHandlerImpl(wbMgr) + fn("ch-0", []int64{1, 2, 3}) + + // test normal + wbMgr = writebuffer.NewMockBufferManager(t) + wbMgr.EXPECT().SealSegments(mock.Anything, mock.Anything, mock.Anything).Return(nil) + + fn = flushMsgHandlerImpl(wbMgr) + fn("ch-0", []int64{1, 2, 3}) +} diff --git a/internal/streamingnode/server/flusher/flusherimpl/pipeline_params.go b/internal/streamingnode/server/flusher/flusherimpl/pipeline_params.go new file mode 100644 index 0000000000..2427d35659 --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/pipeline_params.go @@ -0,0 +1,60 @@ +// 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 flusherimpl + +import ( + "context" + "sync" + + "github.com/milvus-io/milvus/internal/flushcommon/broker" + "github.com/milvus-io/milvus/internal/flushcommon/util" + "github.com/milvus-io/milvus/internal/streamingnode/server/resource" + "github.com/milvus-io/milvus/internal/streamingnode/server/resource/idalloc" + "github.com/milvus-io/milvus/pkg/util/paramtable" +) + +var ( + pipelineParams *util.PipelineParams + initOnce sync.Once +) + +func initPipelineParams() { + initOnce.Do(func() { + var ( + rsc = resource.Resource() + syncMgr = rsc.SyncManager() + wbMgr = rsc.BufferManager() + coordBroker = broker.NewCoordBroker(rsc.DataCoordClient(), paramtable.GetNodeID()) + cpUpdater = util.NewChannelCheckpointUpdater(coordBroker) + ) + pipelineParams = &util.PipelineParams{ + Ctx: context.Background(), + Broker: coordBroker, + SyncMgr: syncMgr, + ChunkManager: rsc.ChunkManager(), + WriteBufferManager: wbMgr, + CheckpointUpdater: cpUpdater, + Allocator: idalloc.NewMAllocator(rsc.IDAllocator()), + FlushMsgHandler: flushMsgHandlerImpl(wbMgr), + } + }) +} + +func GetPipelineParams() *util.PipelineParams { + initPipelineParams() + return pipelineParams +} diff --git a/internal/streamingnode/server/flusher/flushmsg_handler.go b/internal/streamingnode/server/flusher/flushmsg_handler.go new file mode 100644 index 0000000000..00a9b1f42f --- /dev/null +++ b/internal/streamingnode/server/flusher/flushmsg_handler.go @@ -0,0 +1,21 @@ +// 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 flusher + +// TODO: type FlushMsgHandler = func(vchannel string, msg FlushMsg) + +type FlushMsgHandler = func(vchannel string, segmentIDs []int64) diff --git a/internal/streamingnode/server/resource/idalloc/mallocator.go b/internal/streamingnode/server/resource/idalloc/mallocator.go new file mode 100644 index 0000000000..b4b358ecd7 --- /dev/null +++ b/internal/streamingnode/server/resource/idalloc/mallocator.go @@ -0,0 +1,61 @@ +// 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 idalloc + +import ( + "context" + "time" + + "github.com/milvus-io/milvus/internal/allocator" +) + +type mAllocator struct { + allocator Allocator +} + +func NewMAllocator(allocator Allocator) allocator.Interface { + return &mAllocator{allocator: allocator} +} + +func (m *mAllocator) Alloc(count uint32) (int64, int64, error) { + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + var ( + start int64 = 0 + end int64 = 0 + ) + for i := 0; i < int(count)+1; i++ { + id, err := m.allocator.Allocate(ctx) + if err != nil { + return 0, 0, err + } + if i == 0 { + start = int64(id) + } + if i == int(count) { + end = int64(id) + } + } + return start, end, nil +} + +func (m *mAllocator) AllocOne() (int64, error) { + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + id, err := m.allocator.Allocate(ctx) + return int64(id), err +} diff --git a/internal/streamingnode/server/resource/resource.go b/internal/streamingnode/server/resource/resource.go index 5243f78628..409c1cc7cd 100644 --- a/internal/streamingnode/server/resource/resource.go +++ b/internal/streamingnode/server/resource/resource.go @@ -5,6 +5,10 @@ import ( clientv3 "go.etcd.io/etcd/client/v3" + "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" + "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" + "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/internal/streamingnode/server/flusher" "github.com/milvus-io/milvus/internal/streamingnode/server/resource/idalloc" "github.com/milvus-io/milvus/internal/types" ) @@ -14,6 +18,27 @@ var r *resourceImpl // singleton resource instance // optResourceInit is the option to initialize the resource. type optResourceInit func(r *resourceImpl) +// OptFlusher provides the flusher to the resource. +func OptFlusher(flusher flusher.Flusher) optResourceInit { + return func(r *resourceImpl) { + r.flusher = flusher + } +} + +// OptSyncManager provides the sync manager to the resource. +func OptSyncManager(syncMgr syncmgr.SyncManager) optResourceInit { + return func(r *resourceImpl) { + r.syncMgr = syncMgr + } +} + +// OptBufferManager provides the write buffer manager to the resource. +func OptBufferManager(wbMgr writebuffer.BufferManager) optResourceInit { + return func(r *resourceImpl) { + r.wbMgr = wbMgr + } +} + // OptETCD provides the etcd client to the resource. func OptETCD(etcd *clientv3.Client) optResourceInit { return func(r *resourceImpl) { @@ -21,6 +46,13 @@ func OptETCD(etcd *clientv3.Client) optResourceInit { } } +// OptChunkManager provides the chunk manager to the resource. +func OptChunkManager(chunkManager storage.ChunkManager) optResourceInit { + return func(r *resourceImpl) { + r.chunkManager = chunkManager + } +} + // OptRootCoordClient provides the root coordinator client to the resource. func OptRootCoordClient(rootCoordClient types.RootCoordClient) optResourceInit { return func(r *resourceImpl) { @@ -46,7 +78,6 @@ func Init(opts ...optResourceInit) { r.idAllocator = idalloc.NewIDAllocator(r.rootCoordClient) assertNotNil(r.TSOAllocator()) - assertNotNil(r.ETCD()) assertNotNil(r.RootCoordClient()) assertNotNil(r.DataCoordClient()) } @@ -59,13 +90,33 @@ func Resource() *resourceImpl { // resourceImpl is a basic resource dependency for streamingnode server. // All utility on it is concurrent-safe and singleton. type resourceImpl struct { + flusher flusher.Flusher + syncMgr syncmgr.SyncManager + wbMgr writebuffer.BufferManager + timestampAllocator idalloc.Allocator idAllocator idalloc.Allocator etcdClient *clientv3.Client + chunkManager storage.ChunkManager rootCoordClient types.RootCoordClient dataCoordClient types.DataCoordClient } +// Flusher returns the flusher. +func (r *resourceImpl) Flusher() flusher.Flusher { + return r.flusher +} + +// SyncManager returns the sync manager. +func (r *resourceImpl) SyncManager() syncmgr.SyncManager { + return r.syncMgr +} + +// BufferManager returns the write buffer manager. +func (r *resourceImpl) BufferManager() writebuffer.BufferManager { + return r.wbMgr +} + // TSOAllocator returns the timestamp allocator to allocate timestamp. func (r *resourceImpl) TSOAllocator() idalloc.Allocator { return r.timestampAllocator @@ -81,6 +132,11 @@ func (r *resourceImpl) ETCD() *clientv3.Client { return r.etcdClient } +// ChunkManager returns the chunk manager. +func (r *resourceImpl) ChunkManager() storage.ChunkManager { + return r.chunkManager +} + // RootCoordClient returns the root coordinator client. func (r *resourceImpl) RootCoordClient() types.RootCoordClient { return r.rootCoordClient diff --git a/internal/streamingnode/server/wal/adaptor/builder.go b/internal/streamingnode/server/wal/adaptor/builder.go index 4dd1869006..0e8084a4e7 100644 --- a/internal/streamingnode/server/wal/adaptor/builder.go +++ b/internal/streamingnode/server/wal/adaptor/builder.go @@ -3,6 +3,7 @@ package adaptor import ( "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/ddl" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick" "github.com/milvus-io/milvus/pkg/streaming/walimpls" ) @@ -31,5 +32,6 @@ func (b builderAdaptorImpl) Build() (wal.Opener, error) { // Add all interceptor here. return adaptImplsToOpener(o, []interceptors.InterceptorBuilder{ timetick.NewInterceptorBuilder(), + ddl.NewInterceptorBuilder(), }), nil } diff --git a/internal/streamingnode/server/wal/adaptor/message_handler.go b/internal/streamingnode/server/wal/adaptor/message_handler.go index 7dfc7aa6f0..5be230c4f0 100644 --- a/internal/streamingnode/server/wal/adaptor/message_handler.go +++ b/internal/streamingnode/server/wal/adaptor/message_handler.go @@ -15,10 +15,10 @@ import ( type defaultMessageHandler chan message.ImmutableMessage -func (h defaultMessageHandler) Handle(ctx context.Context, upstream <-chan message.ImmutableMessage, msg message.ImmutableMessage) (incoming message.ImmutableMessage, ok bool, err error) { +func (d defaultMessageHandler) Handle(ctx context.Context, upstream <-chan message.ImmutableMessage, msg message.ImmutableMessage) (incoming message.ImmutableMessage, ok bool, err error) { var sendingCh chan message.ImmutableMessage if msg != nil { - sendingCh = h + sendingCh = d } select { case <-ctx.Done(): @@ -125,7 +125,7 @@ func (m *MsgPackAdaptorHandler) addMsgPackIntoPending(msgs ...message.ImmutableM } } -// Close close the handler. +// Close closes the handler. func (m *MsgPackAdaptorHandler) Close() { close(m.channel) } diff --git a/internal/streamingnode/server/wal/interceptors/ddl/builder.go b/internal/streamingnode/server/wal/interceptors/ddl/builder.go new file mode 100644 index 0000000000..1164796313 --- /dev/null +++ b/internal/streamingnode/server/wal/interceptors/ddl/builder.go @@ -0,0 +1,39 @@ +// 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 ddl + +import ( + "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors" +) + +var _ interceptors.InterceptorBuilder = (*interceptorBuilder)(nil) + +// NewInterceptorBuilder creates a new ddl interceptor builder. +func NewInterceptorBuilder() interceptors.InterceptorBuilder { + return &interceptorBuilder{} +} + +// interceptorBuilder is a builder to build ddlAppendInterceptor. +type interceptorBuilder struct{} + +// Build implements Builder. +func (b *interceptorBuilder) Build(param interceptors.InterceptorBuildParam) interceptors.BasicInterceptor { + interceptor := &ddlAppendInterceptor{ + wal: param.WAL, + } + return interceptor +} diff --git a/internal/streamingnode/server/wal/interceptors/ddl/ddl_interceptor.go b/internal/streamingnode/server/wal/interceptors/ddl/ddl_interceptor.go new file mode 100644 index 0000000000..10d989438c --- /dev/null +++ b/internal/streamingnode/server/wal/interceptors/ddl/ddl_interceptor.go @@ -0,0 +1,48 @@ +// 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 ddl + +import ( + "context" + + "github.com/milvus-io/milvus/internal/streamingnode/server/resource" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/util/syncutil" +) + +var _ interceptors.AppendInterceptor = (*ddlAppendInterceptor)(nil) + +// ddlAppendInterceptor is an append interceptor. +type ddlAppendInterceptor struct { + wal *syncutil.Future[wal.WAL] +} + +// DoAppend implements AppendInterceptor. +func (d *ddlAppendInterceptor) DoAppend(ctx context.Context, msg message.MutableMessage, append interceptors.Append) (msgID message.MessageID, err error) { + switch msg.MessageType() { + case message.MessageTypeCreateCollection: + resource.Resource().Flusher().RegisterVChannel(msg.VChannel(), d.wal.Get()) + case message.MessageTypeDropCollection: + resource.Resource().Flusher().UnregisterVChannel(msg.VChannel()) + } + return append(ctx, msg) +} + +// Close implements BasicInterceptor. +func (d *ddlAppendInterceptor) Close() {} diff --git a/internal/streamingnode/server/walmanager/manager_impl_test.go b/internal/streamingnode/server/walmanager/manager_impl_test.go index dbeb8ee026..8907676e15 100644 --- a/internal/streamingnode/server/walmanager/manager_impl_test.go +++ b/internal/streamingnode/server/walmanager/manager_impl_test.go @@ -7,8 +7,11 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" + "github.com/milvus-io/milvus/internal/mocks" + "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_flusher" "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_wal" "github.com/milvus-io/milvus/internal/proto/streamingpb" + "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/streaming/util/types" @@ -21,6 +24,18 @@ func TestMain(m *testing.M) { } func TestManager(t *testing.T) { + rootcoord := mocks.NewMockRootCoordClient(t) + datacoord := mocks.NewMockDataCoordClient(t) + + flusher := mock_flusher.NewMockFlusher(t) + flusher.EXPECT().RegisterPChannel(mock.Anything, mock.Anything).Return(nil) + + resource.Init( + resource.OptFlusher(flusher), + resource.OptRootCoordClient(rootcoord), + resource.OptDataCoordClient(datacoord), + ) + opener := mock_wal.NewMockOpener(t) opener.EXPECT().Open(mock.Anything, mock.Anything).RunAndReturn( func(ctx context.Context, oo *wal.OpenOption) (wal.WAL, error) { diff --git a/internal/streamingnode/server/walmanager/wal_lifetime.go b/internal/streamingnode/server/walmanager/wal_lifetime.go index 96aa7b0301..616c1bc7c4 100644 --- a/internal/streamingnode/server/walmanager/wal_lifetime.go +++ b/internal/streamingnode/server/walmanager/wal_lifetime.go @@ -5,6 +5,7 @@ import ( "go.uber.org/zap" + "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" @@ -17,6 +18,7 @@ func newWALLifetime(opener wal.Opener, channel string) *walLifetime { l := &walLifetime{ ctx: ctx, cancel: cancel, + channel: channel, finish: make(chan struct{}), opener: opener, statePair: newWALStatePair(), @@ -33,8 +35,9 @@ func newWALLifetime(opener wal.Opener, channel string) *walLifetime { // term is always increasing, available is always before unavailable in same term, such as: // (-1, false) -> (0, true) -> (1, true) -> (2, true) -> (3, false) -> (7, true) -> ... type walLifetime struct { - ctx context.Context - cancel context.CancelFunc + ctx context.Context + cancel context.CancelFunc + channel string finish chan struct{} opener wal.Opener @@ -129,7 +132,7 @@ func (w *walLifetime) doLifetimeChanged(expectedState expectedWALState) { // term must be increasing or available -> unavailable, close current term wal is always applied. term := currentState.Term() if oldWAL := currentState.GetWAL(); oldWAL != nil { - // TODO: flusher.Close() + resource.Resource().Flusher().UnregisterPChannel(w.channel) oldWAL.Close() logger.Info("close current term wal done") // Push term to current state unavailable and open a new wal. @@ -149,7 +152,6 @@ func (w *walLifetime) doLifetimeChanged(expectedState expectedWALState) { l, err := w.opener.Open(expectedState.Context(), &wal.OpenOption{ Channel: expectedState.GetPChannelInfo(), }) - // TODO: flusher.Open() if err != nil { logger.Warn("open new wal fail", zap.Error(err)) // Open new wal at expected term failed, push expected term to current state unavailable. @@ -158,6 +160,14 @@ func (w *walLifetime) doLifetimeChanged(expectedState expectedWALState) { return } logger.Info("open new wal done") + err = resource.Resource().Flusher().RegisterPChannel(w.channel, l) + if err != nil { + logger.Warn("open flusher fail", zap.Error(err)) + w.statePair.SetCurrentState(newUnavailableCurrentState(expectedState.Term(), err)) + // wal is opened, if register flusher failure, we should close the wal. + l.Close() + return + } // -> (expectedTerm,true) w.statePair.SetCurrentState(newAvailableCurrentState(l)) } diff --git a/internal/streamingnode/server/walmanager/wal_lifetime_test.go b/internal/streamingnode/server/walmanager/wal_lifetime_test.go index 8d8187f316..7ce53ab393 100644 --- a/internal/streamingnode/server/walmanager/wal_lifetime_test.go +++ b/internal/streamingnode/server/walmanager/wal_lifetime_test.go @@ -7,13 +7,30 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" + "github.com/milvus-io/milvus/internal/mocks" + "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_flusher" "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_wal" + "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/pkg/streaming/util/types" ) func TestWALLifetime(t *testing.T) { channel := "test" + + rootcoord := mocks.NewMockRootCoordClient(t) + datacoord := mocks.NewMockDataCoordClient(t) + + flusher := mock_flusher.NewMockFlusher(t) + flusher.EXPECT().RegisterPChannel(mock.Anything, mock.Anything).Return(nil) + flusher.EXPECT().UnregisterPChannel(mock.Anything).Return() + + resource.Init( + resource.OptFlusher(flusher), + resource.OptRootCoordClient(rootcoord), + resource.OptDataCoordClient(datacoord), + ) + opener := mock_wal.NewMockOpener(t) opener.EXPECT().Open(mock.Anything, mock.Anything).RunAndReturn( func(ctx context.Context, oo *wal.OpenOption) (wal.WAL, error) { diff --git a/pkg/streaming/util/message/adaptor/message_id.go b/pkg/streaming/util/message/adaptor/message_id.go index 645c9c8b2f..538e0ae95e 100644 --- a/pkg/streaming/util/message/adaptor/message_id.go +++ b/pkg/streaming/util/message/adaptor/message_id.go @@ -1,6 +1,8 @@ package adaptor import ( + "fmt" + "github.com/apache/pulsar-client-go/pulsar" "github.com/milvus-io/milvus/pkg/mq/common" @@ -32,3 +34,21 @@ func MustGetMessageIDFromMQWrapperID(commonMessageID common.MessageID) message.M } return nil } + +// DeserializeToMQWrapperID deserializes messageID bytes to common.MessageID +// TODO: should be removed in future after common.MessageID is removed +func DeserializeToMQWrapperID(msgID []byte, walName string) (common.MessageID, error) { + switch walName { + case "pulsar": + pulsarID, err := mqpulsar.DeserializePulsarMsgID(msgID) + if err != nil { + return nil, err + } + return mqpulsar.NewPulsarID(pulsarID), nil + case "rocksmq": + rID := server.DeserializeRmqID(msgID) + return &server.RmqID{MessageID: rID}, nil + default: + return nil, fmt.Errorf("unsupported mq type %s", walName) + } +} diff --git a/pkg/util/paramtable/component_param.go b/pkg/util/paramtable/component_param.go index 09cb245575..3941e28adb 100644 --- a/pkg/util/paramtable/component_param.go +++ b/pkg/util/paramtable/component_param.go @@ -4034,7 +4034,15 @@ func (p *dataNodeConfig) init(base *BaseTable) { Version: "2.3.4", DefaultValue: "256", Doc: "The max concurrent sync task number of datanode sync mgr globally", - Export: true, + Formatter: func(v string) string { + concurrency := getAsInt(v) + if concurrency < 1 { + log.Warn("positive parallel task number, reset to default 256", zap.String("value", v)) + return "256" // MaxParallelSyncMgrTasks must >= 1 + } + return strconv.FormatInt(int64(concurrency), 10) + }, + Export: true, } p.MaxParallelSyncMgrTasks.Init(base.mgr)