mirror of https://github.com/milvus-io/milvus.git
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 <yihao.dai@zilliz.com>pull/35246/head
parent
fcec4c21b9
commit
a4439cc911
22
Makefile
22
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
|
||||
|
|
|
@ -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:
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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)
|
||||
})
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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...)
|
||||
}
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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,
|
||||
},
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}{
|
||||
{
|
||||
"msgTs<segTs",
|
||||
true,
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
map[typeutil.UniqueID]*datapb.SegmentInfo{
|
||||
100: getSegmentInfo(100, 50000),
|
||||
101: getSegmentInfo(101, 50000),
|
||||
},
|
||||
|
@ -492,7 +493,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
|||
{
|
||||
"msgTs==segTs",
|
||||
true,
|
||||
map[util.UniqueID]*datapb.SegmentInfo{
|
||||
map[typeutil.UniqueID]*datapb.SegmentInfo{
|
||||
100: getSegmentInfo(100, 50000),
|
||||
101: getSegmentInfo(101, 50000),
|
||||
},
|
||||
|
@ -502,7 +503,7 @@ func TestFlowGraph_DDNode_filterMessages(t *testing.T) {
|
|||
{
|
||||
"msgTs>segTs",
|
||||
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{
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
}{
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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,
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
|
@ -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)
|
||||
}
|
|
@ -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
|
|
@ -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)
|
||||
})
|
||||
}
|
|
@ -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
|
|
@ -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"
|
|
@ -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) {
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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)}
|
||||
}
|
||||
|
|
|
@ -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...)...)}
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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()
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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))
|
||||
}
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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})
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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)
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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() {}
|
|
@ -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) {
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
|
Loading…
Reference in New Issue