diff --git a/internal/.mockery.yaml b/internal/.mockery.yaml index 8804105c23322..4be007c939464 100644 --- a/internal/.mockery.yaml +++ b/internal/.mockery.yaml @@ -36,7 +36,6 @@ packages: Consumer: github.com/milvus-io/milvus/internal/streamingnode/server/flusher: interfaces: - Flusher: FlushMsgHandler: github.com/milvus-io/milvus/internal/streamingnode/server/wal: interfaces: @@ -55,6 +54,9 @@ packages: github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick/inspector: interfaces: TimeTickSyncOperator: + github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/wab: + interfaces: + ROWriteAheadBuffer: google.golang.org/grpc: interfaces: ClientStream: @@ -68,6 +70,9 @@ packages: github.com/milvus-io/milvus/internal/util/segcore: interfaces: CSegment: + github.com/milvus-io/milvus/internal/storage: + interfaces: + ChunkManager: github.com/milvus-io/milvus/internal/util/streamingutil/service/discoverer: interfaces: Discoverer: diff --git a/internal/distributed/streaming/internal/consumer/consumer_impl.go b/internal/distributed/streaming/internal/consumer/consumer_impl.go index 082c61afd1758..14632e3187ab9 100644 --- a/internal/distributed/streaming/internal/consumer/consumer_impl.go +++ b/internal/distributed/streaming/internal/consumer/consumer_impl.go @@ -74,11 +74,12 @@ func (rc *resumableConsumerImpl) resumeLoop() { // consumer need to resume when error occur, so message handler shouldn't close if the internal consumer encounter failure. nopCloseMH := nopCloseHandler{ Handler: rc.mh, - HandleInterceptor: func(ctx context.Context, msg message.ImmutableMessage, handle handleFunc) (bool, error) { - g := rc.metrics.StartConsume(msg.EstimateSize()) - ok, err := handle(ctx, msg) - g.Finish() - return ok, err + HandleInterceptor: func(handleParam message.HandleParam, h message.Handler) message.HandleResult { + if handleParam.Message != nil { + g := rc.metrics.StartConsume(handleParam.Message.EstimateSize()) + defer func() { g.Finish() }() + } + return h.Handle(handleParam) }, } diff --git a/internal/distributed/streaming/internal/consumer/consumer_test.go b/internal/distributed/streaming/internal/consumer/consumer_test.go index c6a85792b320c..ffe4d01177de2 100644 --- a/internal/distributed/streaming/internal/consumer/consumer_test.go +++ b/internal/distributed/streaming/internal/consumer/consumer_test.go @@ -12,6 +12,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/client/handler" "github.com/milvus-io/milvus/internal/streamingnode/client/handler/consumer" "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/streaming/walimpls/impls/walimplstest" ) @@ -22,22 +23,25 @@ func TestResumableConsumer(t *testing.T) { ch := make(chan struct{}) c.EXPECT().Done().Return(ch) c.EXPECT().Error().Return(errors.New("test")) - c.EXPECT().Close().Return() + c.EXPECT().Close().Return(nil) rc := NewResumableConsumer(func(ctx context.Context, opts *handler.ConsumerOptions) (consumer.Consumer, error) { if i == 0 { i++ - ok, err := opts.MessageHandler.Handle(context.Background(), message.NewImmutableMesasge( - walimplstest.NewTestMessageID(123), - []byte("payload"), - map[string]string{ - "key": "value", - "_t": "1", - "_tt": message.EncodeUint64(456), - "_v": "1", - "_lc": walimplstest.NewTestMessageID(123).Marshal(), - })) - assert.True(t, ok) - assert.NoError(t, err) + result := opts.MessageHandler.Handle(message.HandleParam{ + Ctx: context.Background(), + Message: message.NewImmutableMesasge( + walimplstest.NewTestMessageID(123), + []byte("payload"), + map[string]string{ + "key": "value", + "_t": "1", + "_tt": message.EncodeUint64(456), + "_v": "1", + "_lc": walimplstest.NewTestMessageID(123).Marshal(), + }), + }) + assert.True(t, result.MessageHandled) + assert.NoError(t, result.Error) return c, nil } else if i == 1 { i++ @@ -46,7 +50,7 @@ func TestResumableConsumer(t *testing.T) { newC := mock_consumer.NewMockConsumer(t) newC.EXPECT().Done().Return(make(<-chan struct{})) newC.EXPECT().Error().Return(errors.New("test")) - newC.EXPECT().Close().Return() + newC.EXPECT().Close().Return(nil) return newC, nil }, &ConsumerOptions{ PChannel: "test", @@ -54,7 +58,7 @@ func TestResumableConsumer(t *testing.T) { DeliverFilters: []options.DeliverFilter{ options.DeliverFilterTimeTickGT(1), }, - MessageHandler: message.ChanMessageHandler(make(chan message.ImmutableMessage, 2)), + MessageHandler: adaptor.ChanMessageHandler(make(chan message.ImmutableMessage, 2)), }) select { @@ -76,10 +80,13 @@ func TestResumableConsumer(t *testing.T) { func TestHandler(t *testing.T) { ch := make(chan message.ImmutableMessage, 100) hNop := nopCloseHandler{ - Handler: message.ChanMessageHandler(ch), + Handler: adaptor.ChanMessageHandler(ch), } - hNop.Handle(context.Background(), nil) - assert.Nil(t, <-ch) + hNop.Handle(message.HandleParam{ + Ctx: context.Background(), + Message: message.NewImmutableMesasge(walimplstest.NewTestMessageID(123), []byte("payload"), nil), + }) + assert.NotNil(t, <-ch) hNop.Close() select { case <-ch: diff --git a/internal/distributed/streaming/internal/consumer/handler.go b/internal/distributed/streaming/internal/consumer/handler.go index d106b9da4d5fe..c78bbd2de826d 100644 --- a/internal/distributed/streaming/internal/consumer/handler.go +++ b/internal/distributed/streaming/internal/consumer/handler.go @@ -1,25 +1,21 @@ package consumer import ( - "context" - "github.com/milvus-io/milvus/pkg/streaming/util/message" ) -type handleFunc func(ctx context.Context, msg message.ImmutableMessage) (bool, error) - // nopCloseHandler is a handler that do nothing when close. type nopCloseHandler struct { message.Handler - HandleInterceptor func(ctx context.Context, msg message.ImmutableMessage, handle handleFunc) (bool, error) + HandleInterceptor func(handleParam message.HandleParam, h message.Handler) message.HandleResult } // Handle is the callback for handling message. -func (nch nopCloseHandler) Handle(ctx context.Context, msg message.ImmutableMessage) (bool, error) { +func (nch nopCloseHandler) Handle(handleParam message.HandleParam) message.HandleResult { if nch.HandleInterceptor != nil { - return nch.HandleInterceptor(ctx, msg, nch.Handler.Handle) + return nch.HandleInterceptor(handleParam, nch.Handler) } - return nch.Handler.Handle(ctx, msg) + return nch.Handler.Handle(handleParam) } // Close is called after all messages are handled or handling is interrupted. diff --git a/internal/distributed/streaming/internal/consumer/message_handler.go b/internal/distributed/streaming/internal/consumer/message_handler.go index 538052ee174c0..e790ad2c6d773 100644 --- a/internal/distributed/streaming/internal/consumer/message_handler.go +++ b/internal/distributed/streaming/internal/consumer/message_handler.go @@ -1,8 +1,6 @@ package consumer import ( - "context" - "github.com/milvus-io/milvus/pkg/streaming/util/message" ) @@ -13,16 +11,20 @@ type timeTickOrderMessageHandler struct { lastTimeTick uint64 } -func (mh *timeTickOrderMessageHandler) Handle(ctx context.Context, msg message.ImmutableMessage) (bool, error) { - lastConfirmedMessageID := msg.LastConfirmedMessageID() - timetick := msg.TimeTick() +func (mh *timeTickOrderMessageHandler) Handle(handleParam message.HandleParam) message.HandleResult { + var lastConfirmedMessageID message.MessageID + var lastTimeTick uint64 + if handleParam.Message != nil { + lastConfirmedMessageID = handleParam.Message.LastConfirmedMessageID() + lastTimeTick = handleParam.Message.TimeTick() + } - ok, err := mh.inner.Handle(ctx, msg) - if ok { + result := mh.inner.Handle(handleParam) + if result.MessageHandled { mh.lastConfirmedMessageID = lastConfirmedMessageID - mh.lastTimeTick = timetick + mh.lastTimeTick = lastTimeTick } - return ok, err + return result } func (mh *timeTickOrderMessageHandler) Close() { diff --git a/internal/distributed/streaming/internal/producer/producer.go b/internal/distributed/streaming/internal/producer/producer.go index 1feaab90a6fdf..19eb0aaef8527 100644 --- a/internal/distributed/streaming/internal/producer/producer.go +++ b/internal/distributed/streaming/internal/producer/producer.go @@ -15,6 +15,7 @@ import ( "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/syncutil" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -77,7 +78,7 @@ type ResumableProducer struct { } // Produce produce a new message to log service. -func (p *ResumableProducer) Produce(ctx context.Context, msg message.MutableMessage) (result *producer.ProduceResult, err error) { +func (p *ResumableProducer) Produce(ctx context.Context, msg message.MutableMessage) (result *types.AppendResult, err error) { if !p.lifetime.Add(typeutil.LifetimeStateWorking) { return nil, errors.Wrapf(errs.ErrClosed, "produce on closed producer") } @@ -94,7 +95,7 @@ func (p *ResumableProducer) Produce(ctx context.Context, msg message.MutableMess return nil, err } - produceResult, err := producerHandler.Produce(ctx, msg) + produceResult, err := producerHandler.Append(ctx, msg) if err == nil { return produceResult, nil } diff --git a/internal/distributed/streaming/internal/producer/producer_test.go b/internal/distributed/streaming/internal/producer/producer_test.go index d98be5dde3d32..7ef50b7fc2ec4 100644 --- a/internal/distributed/streaming/internal/producer/producer_test.go +++ b/internal/distributed/streaming/internal/producer/producer_test.go @@ -14,12 +14,13 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/client/handler/producer" "github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message" "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/types" ) func TestResumableProducer(t *testing.T) { p := mock_producer.NewMockProducer(t) msgID := mock_message.NewMockMessageID(t) - p.EXPECT().Produce(mock.Anything, mock.Anything).Return(&producer.ProduceResult{ + p.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.AppendResult{ MessageID: msgID, TimeTick: 100, }, nil) @@ -47,11 +48,11 @@ func TestResumableProducer(t *testing.T) { } else if i == 2 { p := mock_producer.NewMockProducer(t) msgID := mock_message.NewMockMessageID(t) - p.EXPECT().Produce(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, mm message.MutableMessage) (*producer.ProduceResult, error) { + p.EXPECT().Append(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, mm message.MutableMessage) (*types.AppendResult, error) { if ctx.Err() != nil { return nil, ctx.Err() } - return &producer.ProduceResult{ + return &types.AppendResult{ MessageID: msgID, TimeTick: 100, }, nil diff --git a/internal/distributed/streaming/streaming_test.go b/internal/distributed/streaming/streaming_test.go index 0da14f0923512..55ffd917d766a 100644 --- a/internal/distributed/streaming/streaming_test.go +++ b/internal/distributed/streaming/streaming_test.go @@ -10,6 +10,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/distributed/streaming" "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/streaming/walimpls/impls/pulsar" "github.com/milvus-io/milvus/pkg/util/paramtable" @@ -109,7 +110,7 @@ func TestStreamingConsume(t *testing.T) { t.Skip() streaming.Init() defer streaming.Release() - ch := make(message.ChanMessageHandler, 10) + ch := make(adaptor.ChanMessageHandler, 10) s := streaming.WAL().Read(context.Background(), streaming.ReadOption{ VChannel: vChannels[0], DeliverPolicy: options.DeliverPolicyAll(), diff --git a/internal/distributed/streaming/wal_test.go b/internal/distributed/streaming/wal_test.go index a850b9cce3a07..71ee775cbaa17 100644 --- a/internal/distributed/streaming/wal_test.go +++ b/internal/distributed/streaming/wal_test.go @@ -70,7 +70,7 @@ func TestWAL(t *testing.T) { return true } }) - p.EXPECT().Produce(mock.Anything, mock.Anything).Return(&types.AppendResult{ + p.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.AppendResult{ MessageID: walimplstest.NewTestMessageID(1), TimeTick: 10, TxnCtx: &message.TxnContext{ diff --git a/internal/distributed/streamingnode/service.go b/internal/distributed/streamingnode/service.go index a3d3e04268343..4464eb9c3245e 100644 --- a/internal/distributed/streamingnode/service.go +++ b/internal/distributed/streamingnode/service.go @@ -242,7 +242,6 @@ func (s *Server) init() (err error) { WithDataCoordClient(s.dataCoord). WithSession(s.session). WithMetaKV(s.metaKV). - WithChunkManager(s.chunkManager). Build() if err := s.streamingnode.Init(s.ctx); err != nil { return errors.Wrap(err, "StreamingNode service init failed") diff --git a/internal/flushcommon/pipeline/data_sync_service.go b/internal/flushcommon/pipeline/data_sync_service.go index b9f23475b8f02..23cbb30ef8afa 100644 --- a/internal/flushcommon/pipeline/data_sync_service.go +++ b/internal/flushcommon/pipeline/data_sync_service.go @@ -18,10 +18,12 @@ package pipeline import ( "context" + "fmt" "sync" "go.uber.org/zap" + "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/io" @@ -407,3 +409,27 @@ func NewStreamingNodeDataSyncService( func NewDataSyncServiceWithMetaCache(metaCache metacache.MetaCache) *DataSyncService { return &DataSyncService{metacache: metaCache} } + +func NewEmptyStreamingNodeDataSyncService( + initCtx context.Context, + pipelineParams *util.PipelineParams, + input <-chan *msgstream.MsgPack, + vchannelInfo *datapb.VchannelInfo, + schema *schemapb.CollectionSchema, + wbTaskObserverCallback writebuffer.TaskObserverCallback, + dropCallback func(), +) *DataSyncService { + watchInfo := &datapb.ChannelWatchInfo{ + Vchan: vchannelInfo, + Schema: schema, + } + metaCache, err := getMetaCacheForStreaming(initCtx, pipelineParams, watchInfo, make([]*datapb.SegmentInfo, 0), make([]*datapb.SegmentInfo, 0)) + if err != nil { + panic(fmt.Sprintf("new a empty streaming node data sync service should never be failed, %s", err.Error())) + } + ds, err := getServiceWithChannel(initCtx, pipelineParams, watchInfo, metaCache, make([]*datapb.SegmentInfo, 0), make([]*datapb.SegmentInfo, 0), input, wbTaskObserverCallback, dropCallback) + if err != nil { + panic(fmt.Sprintf("new a empty data sync service should never be failed, %s", err.Error())) + } + return ds +} diff --git a/internal/flushcommon/syncmgr/sync_manager.go b/internal/flushcommon/syncmgr/sync_manager.go index da0ee0e99a0da..fe6162d4dc4c3 100644 --- a/internal/flushcommon/syncmgr/sync_manager.go +++ b/internal/flushcommon/syncmgr/sync_manager.go @@ -60,6 +60,7 @@ type syncManager struct { tasks *typeutil.ConcurrentMap[string, Task] taskStats *expirable.LRU[string, Task] + handler config.EventHandler } func NewSyncManager(chunkManager storage.ChunkManager) SyncManager { @@ -75,7 +76,9 @@ func NewSyncManager(chunkManager storage.ChunkManager) SyncManager { taskStats: expirable.NewLRU[string, Task](16, nil, time.Minute*15), } // setup config update watcher - params.Watch(params.DataNodeCfg.MaxParallelSyncMgrTasks.Key, config.NewHandler("datanode.syncmgr.poolsize", syncMgr.resizeHandler)) + handler := config.NewHandler("datanode.syncmgr.poolsize", syncMgr.resizeHandler) + syncMgr.handler = handler + params.Watch(params.DataNodeCfg.MaxParallelSyncMgrTasks.Key, handler) return syncMgr } @@ -155,6 +158,7 @@ func (mgr *syncManager) TaskStatsJSON() string { } func (mgr *syncManager) Close() error { + paramtable.Get().Unwatch(paramtable.Get().DataNodeCfg.MaxParallelSyncMgrTasks.Key, mgr.handler) timeout := paramtable.Get().CommonCfg.SyncTaskPoolReleaseTimeoutSeconds.GetAsDuration(time.Second) return mgr.workerPool.ReleaseTimeout(timeout) } diff --git a/internal/metastore/catalog.go b/internal/metastore/catalog.go index a8ef2dcaab80d..77cdcd68bc3ea 100644 --- a/internal/metastore/catalog.go +++ b/internal/metastore/catalog.go @@ -231,4 +231,11 @@ type StreamingNodeCataLog interface { // SaveSegmentAssignments save the segment assignments for the wal. SaveSegmentAssignments(ctx context.Context, pChannelName string, infos []*streamingpb.SegmentAssignmentMeta) error + + // GetConsumeCheckpoint gets the consuming checkpoint of the wal. + // Return nil, nil if the checkpoint is not exist. + GetConsumeCheckpoint(ctx context.Context, pChannelName string) (*streamingpb.WALCheckpoint, error) + + // SaveConsumeCheckpoint saves the consuming checkpoint of the wal. + SaveConsumeCheckpoint(ctx context.Context, pChannelName string, checkpoint *streamingpb.WALCheckpoint) error } diff --git a/internal/metastore/kv/streamingnode/constant.go b/internal/metastore/kv/streamingnode/constant.go index 1c83ba17432b2..3d564ceab7f0d 100644 --- a/internal/metastore/kv/streamingnode/constant.go +++ b/internal/metastore/kv/streamingnode/constant.go @@ -5,4 +5,6 @@ const ( DirectoryWAL = "wal" DirectorySegmentAssign = "segment-assign" + + KeyConsumeCheckpoint = "consume-checkpoint" ) diff --git a/internal/metastore/kv/streamingnode/kv_catalog.go b/internal/metastore/kv/streamingnode/kv_catalog.go index 7cb546fdf2a6d..9ba73784df727 100644 --- a/internal/metastore/kv/streamingnode/kv_catalog.go +++ b/internal/metastore/kv/streamingnode/kv_catalog.go @@ -13,6 +13,7 @@ import ( "github.com/milvus-io/milvus/pkg/proto/streamingpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/etcd" + "github.com/milvus-io/milvus/pkg/util/merr" ) // NewCataLog creates a new streaming-node catalog instance. @@ -22,11 +23,13 @@ import ( // └── wal // // ├── pchannel-1 +// │   ├── checkpoint // │   └── segment-assign // │   ├── 456398247934 // │   ├── 456398247936 // │   └── 456398247939 // └── pchannel-2 +// ├── checkpoint // └── segment-assign // ├── 456398247934 // ├── 456398247935 @@ -96,6 +99,33 @@ func (c *catalog) SaveSegmentAssignments(ctx context.Context, pChannelName strin return nil } +// GetConsumeCheckpoint gets the consuming checkpoint of the wal. +func (c *catalog) GetConsumeCheckpoint(ctx context.Context, pchannelName string) (*streamingpb.WALCheckpoint, error) { + key := buildConsumeCheckpointPath(pchannelName) + value, err := c.metaKV.Load(ctx, key) + if errors.Is(err, merr.ErrIoKeyNotFound) { + return nil, nil + } + if err != nil { + return nil, err + } + val := &streamingpb.WALCheckpoint{} + if err = proto.Unmarshal([]byte(value), &streamingpb.WALCheckpoint{}); err != nil { + return nil, err + } + return val, nil +} + +// SaveConsumeCheckpoint saves the consuming checkpoint of the wal. +func (c *catalog) SaveConsumeCheckpoint(ctx context.Context, pchannelName string, checkpoint *streamingpb.WALCheckpoint) error { + key := buildConsumeCheckpointPath(pchannelName) + value, err := proto.Marshal(checkpoint) + if err != nil { + return err + } + return c.metaKV.Save(ctx, key, string(value)) +} + // buildSegmentAssignmentMetaPath builds the path for segment assignment func buildSegmentAssignmentMetaPath(pChannelName string) string { return path.Join(buildWALDirectory(pChannelName), DirectorySegmentAssign) + "/" @@ -106,6 +136,11 @@ func buildSegmentAssignmentMetaPathOfSegment(pChannelName string, segmentID int6 return path.Join(buildWALDirectory(pChannelName), DirectorySegmentAssign, strconv.FormatInt(segmentID, 10)) } +// buildConsumeCheckpointPath builds the path for consume checkpoint +func buildConsumeCheckpointPath(pchannelName string) string { + return path.Join(buildWALDirectory(pchannelName), KeyConsumeCheckpoint) +} + // buildWALDirectory builds the path for wal directory func buildWALDirectory(pchannelName string) string { return path.Join(MetaPrefix, DirectoryWAL, pchannelName) + "/" diff --git a/internal/mocks/distributed/mock_streaming/mock_WALAccesser.go b/internal/mocks/distributed/mock_streaming/mock_WALAccesser.go index eb9f7ce2d8b0a..a3373b8b832a4 100644 --- a/internal/mocks/distributed/mock_streaming/mock_WALAccesser.go +++ b/internal/mocks/distributed/mock_streaming/mock_WALAccesser.go @@ -27,7 +27,7 @@ func (_m *MockWALAccesser) EXPECT() *MockWALAccesser_Expecter { } // AppendMessages provides a mock function with given fields: ctx, msgs -func (_m *MockWALAccesser) AppendMessages(ctx context.Context, msgs ...message.MutableMessage) streaming.AppendResponses { +func (_m *MockWALAccesser) AppendMessages(ctx context.Context, msgs ...message.MutableMessage) types.AppendResponses { _va := make([]interface{}, len(msgs)) for _i := range msgs { _va[_i] = msgs[_i] @@ -41,11 +41,11 @@ func (_m *MockWALAccesser) AppendMessages(ctx context.Context, msgs ...message.M panic("no return value specified for AppendMessages") } - var r0 streaming.AppendResponses - if rf, ok := ret.Get(0).(func(context.Context, ...message.MutableMessage) streaming.AppendResponses); ok { + var r0 types.AppendResponses + if rf, ok := ret.Get(0).(func(context.Context, ...message.MutableMessage) types.AppendResponses); ok { r0 = rf(ctx, msgs...) } else { - r0 = ret.Get(0).(streaming.AppendResponses) + r0 = ret.Get(0).(types.AppendResponses) } return r0 @@ -77,18 +77,18 @@ func (_c *MockWALAccesser_AppendMessages_Call) Run(run func(ctx context.Context, return _c } -func (_c *MockWALAccesser_AppendMessages_Call) Return(_a0 streaming.AppendResponses) *MockWALAccesser_AppendMessages_Call { +func (_c *MockWALAccesser_AppendMessages_Call) Return(_a0 types.AppendResponses) *MockWALAccesser_AppendMessages_Call { _c.Call.Return(_a0) return _c } -func (_c *MockWALAccesser_AppendMessages_Call) RunAndReturn(run func(context.Context, ...message.MutableMessage) streaming.AppendResponses) *MockWALAccesser_AppendMessages_Call { +func (_c *MockWALAccesser_AppendMessages_Call) RunAndReturn(run func(context.Context, ...message.MutableMessage) types.AppendResponses) *MockWALAccesser_AppendMessages_Call { _c.Call.Return(run) return _c } // AppendMessagesWithOption provides a mock function with given fields: ctx, opts, msgs -func (_m *MockWALAccesser) AppendMessagesWithOption(ctx context.Context, opts streaming.AppendOption, msgs ...message.MutableMessage) streaming.AppendResponses { +func (_m *MockWALAccesser) AppendMessagesWithOption(ctx context.Context, opts streaming.AppendOption, msgs ...message.MutableMessage) types.AppendResponses { _va := make([]interface{}, len(msgs)) for _i := range msgs { _va[_i] = msgs[_i] @@ -102,11 +102,11 @@ func (_m *MockWALAccesser) AppendMessagesWithOption(ctx context.Context, opts st panic("no return value specified for AppendMessagesWithOption") } - var r0 streaming.AppendResponses - if rf, ok := ret.Get(0).(func(context.Context, streaming.AppendOption, ...message.MutableMessage) streaming.AppendResponses); ok { + var r0 types.AppendResponses + if rf, ok := ret.Get(0).(func(context.Context, streaming.AppendOption, ...message.MutableMessage) types.AppendResponses); ok { r0 = rf(ctx, opts, msgs...) } else { - r0 = ret.Get(0).(streaming.AppendResponses) + r0 = ret.Get(0).(types.AppendResponses) } return r0 @@ -139,12 +139,12 @@ func (_c *MockWALAccesser_AppendMessagesWithOption_Call) Run(run func(ctx contex return _c } -func (_c *MockWALAccesser_AppendMessagesWithOption_Call) Return(_a0 streaming.AppendResponses) *MockWALAccesser_AppendMessagesWithOption_Call { +func (_c *MockWALAccesser_AppendMessagesWithOption_Call) Return(_a0 types.AppendResponses) *MockWALAccesser_AppendMessagesWithOption_Call { _c.Call.Return(_a0) return _c } -func (_c *MockWALAccesser_AppendMessagesWithOption_Call) RunAndReturn(run func(context.Context, streaming.AppendOption, ...message.MutableMessage) streaming.AppendResponses) *MockWALAccesser_AppendMessagesWithOption_Call { +func (_c *MockWALAccesser_AppendMessagesWithOption_Call) RunAndReturn(run func(context.Context, streaming.AppendOption, ...message.MutableMessage) types.AppendResponses) *MockWALAccesser_AppendMessagesWithOption_Call { _c.Call.Return(run) return _c } diff --git a/internal/mocks/mock_metastore/mock_StreamingNodeCataLog.go b/internal/mocks/mock_metastore/mock_StreamingNodeCataLog.go index ed3f6d6af42d2..f40c0a4fc1b4d 100644 --- a/internal/mocks/mock_metastore/mock_StreamingNodeCataLog.go +++ b/internal/mocks/mock_metastore/mock_StreamingNodeCataLog.go @@ -23,6 +23,65 @@ func (_m *MockStreamingNodeCataLog) EXPECT() *MockStreamingNodeCataLog_Expecter return &MockStreamingNodeCataLog_Expecter{mock: &_m.Mock} } +// GetConsumeCheckpoint provides a mock function with given fields: ctx, pChannelName +func (_m *MockStreamingNodeCataLog) GetConsumeCheckpoint(ctx context.Context, pChannelName string) (*streamingpb.WALCheckpoint, error) { + ret := _m.Called(ctx, pChannelName) + + if len(ret) == 0 { + panic("no return value specified for GetConsumeCheckpoint") + } + + var r0 *streamingpb.WALCheckpoint + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, string) (*streamingpb.WALCheckpoint, error)); ok { + return rf(ctx, pChannelName) + } + if rf, ok := ret.Get(0).(func(context.Context, string) *streamingpb.WALCheckpoint); ok { + r0 = rf(ctx, pChannelName) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*streamingpb.WALCheckpoint) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, string) error); ok { + r1 = rf(ctx, pChannelName) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockStreamingNodeCataLog_GetConsumeCheckpoint_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetConsumeCheckpoint' +type MockStreamingNodeCataLog_GetConsumeCheckpoint_Call struct { + *mock.Call +} + +// GetConsumeCheckpoint is a helper method to define mock.On call +// - ctx context.Context +// - pChannelName string +func (_e *MockStreamingNodeCataLog_Expecter) GetConsumeCheckpoint(ctx interface{}, pChannelName interface{}) *MockStreamingNodeCataLog_GetConsumeCheckpoint_Call { + return &MockStreamingNodeCataLog_GetConsumeCheckpoint_Call{Call: _e.mock.On("GetConsumeCheckpoint", ctx, pChannelName)} +} + +func (_c *MockStreamingNodeCataLog_GetConsumeCheckpoint_Call) Run(run func(ctx context.Context, pChannelName string)) *MockStreamingNodeCataLog_GetConsumeCheckpoint_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string)) + }) + return _c +} + +func (_c *MockStreamingNodeCataLog_GetConsumeCheckpoint_Call) Return(_a0 *streamingpb.WALCheckpoint, _a1 error) *MockStreamingNodeCataLog_GetConsumeCheckpoint_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockStreamingNodeCataLog_GetConsumeCheckpoint_Call) RunAndReturn(run func(context.Context, string) (*streamingpb.WALCheckpoint, error)) *MockStreamingNodeCataLog_GetConsumeCheckpoint_Call { + _c.Call.Return(run) + return _c +} + // ListSegmentAssignment provides a mock function with given fields: ctx, pChannelName func (_m *MockStreamingNodeCataLog) ListSegmentAssignment(ctx context.Context, pChannelName string) ([]*streamingpb.SegmentAssignmentMeta, error) { ret := _m.Called(ctx, pChannelName) @@ -82,6 +141,54 @@ func (_c *MockStreamingNodeCataLog_ListSegmentAssignment_Call) RunAndReturn(run return _c } +// SaveConsumeCheckpoint provides a mock function with given fields: ctx, pChannelName, checkpoint +func (_m *MockStreamingNodeCataLog) SaveConsumeCheckpoint(ctx context.Context, pChannelName string, checkpoint *streamingpb.WALCheckpoint) error { + ret := _m.Called(ctx, pChannelName, checkpoint) + + if len(ret) == 0 { + panic("no return value specified for SaveConsumeCheckpoint") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, string, *streamingpb.WALCheckpoint) error); ok { + r0 = rf(ctx, pChannelName, checkpoint) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockStreamingNodeCataLog_SaveConsumeCheckpoint_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SaveConsumeCheckpoint' +type MockStreamingNodeCataLog_SaveConsumeCheckpoint_Call struct { + *mock.Call +} + +// SaveConsumeCheckpoint is a helper method to define mock.On call +// - ctx context.Context +// - pChannelName string +// - checkpoint *streamingpb.WALCheckpoint +func (_e *MockStreamingNodeCataLog_Expecter) SaveConsumeCheckpoint(ctx interface{}, pChannelName interface{}, checkpoint interface{}) *MockStreamingNodeCataLog_SaveConsumeCheckpoint_Call { + return &MockStreamingNodeCataLog_SaveConsumeCheckpoint_Call{Call: _e.mock.On("SaveConsumeCheckpoint", ctx, pChannelName, checkpoint)} +} + +func (_c *MockStreamingNodeCataLog_SaveConsumeCheckpoint_Call) Run(run func(ctx context.Context, pChannelName string, checkpoint *streamingpb.WALCheckpoint)) *MockStreamingNodeCataLog_SaveConsumeCheckpoint_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string), args[2].(*streamingpb.WALCheckpoint)) + }) + return _c +} + +func (_c *MockStreamingNodeCataLog_SaveConsumeCheckpoint_Call) Return(_a0 error) *MockStreamingNodeCataLog_SaveConsumeCheckpoint_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockStreamingNodeCataLog_SaveConsumeCheckpoint_Call) RunAndReturn(run func(context.Context, string, *streamingpb.WALCheckpoint) error) *MockStreamingNodeCataLog_SaveConsumeCheckpoint_Call { + _c.Call.Return(run) + return _c +} + // SaveSegmentAssignments provides a mock function with given fields: ctx, pChannelName, infos func (_m *MockStreamingNodeCataLog) SaveSegmentAssignments(ctx context.Context, pChannelName string, infos []*streamingpb.SegmentAssignmentMeta) error { ret := _m.Called(ctx, pChannelName, infos) diff --git a/internal/mocks/mock_storage/mock_ChunkManager.go b/internal/mocks/mock_storage/mock_ChunkManager.go new file mode 100644 index 0000000000000..8ac4ce03f3f69 --- /dev/null +++ b/internal/mocks/mock_storage/mock_ChunkManager.go @@ -0,0 +1,838 @@ +// Code generated by mockery v2.46.0. DO NOT EDIT. + +package mock_storage + +import ( + context "context" + + mmap "golang.org/x/exp/mmap" + + mock "github.com/stretchr/testify/mock" + + storage "github.com/milvus-io/milvus/internal/storage" +) + +// MockChunkManager is an autogenerated mock type for the ChunkManager type +type MockChunkManager struct { + mock.Mock +} + +type MockChunkManager_Expecter struct { + mock *mock.Mock +} + +func (_m *MockChunkManager) EXPECT() *MockChunkManager_Expecter { + return &MockChunkManager_Expecter{mock: &_m.Mock} +} + +// Exist provides a mock function with given fields: ctx, filePath +func (_m *MockChunkManager) Exist(ctx context.Context, filePath string) (bool, error) { + ret := _m.Called(ctx, filePath) + + if len(ret) == 0 { + panic("no return value specified for Exist") + } + + var r0 bool + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, string) (bool, error)); ok { + return rf(ctx, filePath) + } + if rf, ok := ret.Get(0).(func(context.Context, string) bool); ok { + r0 = rf(ctx, filePath) + } else { + r0 = ret.Get(0).(bool) + } + + if rf, ok := ret.Get(1).(func(context.Context, string) error); ok { + r1 = rf(ctx, filePath) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockChunkManager_Exist_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Exist' +type MockChunkManager_Exist_Call struct { + *mock.Call +} + +// Exist is a helper method to define mock.On call +// - ctx context.Context +// - filePath string +func (_e *MockChunkManager_Expecter) Exist(ctx interface{}, filePath interface{}) *MockChunkManager_Exist_Call { + return &MockChunkManager_Exist_Call{Call: _e.mock.On("Exist", ctx, filePath)} +} + +func (_c *MockChunkManager_Exist_Call) Run(run func(ctx context.Context, filePath string)) *MockChunkManager_Exist_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string)) + }) + return _c +} + +func (_c *MockChunkManager_Exist_Call) Return(_a0 bool, _a1 error) *MockChunkManager_Exist_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockChunkManager_Exist_Call) RunAndReturn(run func(context.Context, string) (bool, error)) *MockChunkManager_Exist_Call { + _c.Call.Return(run) + return _c +} + +// Mmap provides a mock function with given fields: ctx, filePath +func (_m *MockChunkManager) Mmap(ctx context.Context, filePath string) (*mmap.ReaderAt, error) { + ret := _m.Called(ctx, filePath) + + if len(ret) == 0 { + panic("no return value specified for Mmap") + } + + var r0 *mmap.ReaderAt + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, string) (*mmap.ReaderAt, error)); ok { + return rf(ctx, filePath) + } + if rf, ok := ret.Get(0).(func(context.Context, string) *mmap.ReaderAt); ok { + r0 = rf(ctx, filePath) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*mmap.ReaderAt) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, string) error); ok { + r1 = rf(ctx, filePath) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockChunkManager_Mmap_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Mmap' +type MockChunkManager_Mmap_Call struct { + *mock.Call +} + +// Mmap is a helper method to define mock.On call +// - ctx context.Context +// - filePath string +func (_e *MockChunkManager_Expecter) Mmap(ctx interface{}, filePath interface{}) *MockChunkManager_Mmap_Call { + return &MockChunkManager_Mmap_Call{Call: _e.mock.On("Mmap", ctx, filePath)} +} + +func (_c *MockChunkManager_Mmap_Call) Run(run func(ctx context.Context, filePath string)) *MockChunkManager_Mmap_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string)) + }) + return _c +} + +func (_c *MockChunkManager_Mmap_Call) Return(_a0 *mmap.ReaderAt, _a1 error) *MockChunkManager_Mmap_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockChunkManager_Mmap_Call) RunAndReturn(run func(context.Context, string) (*mmap.ReaderAt, error)) *MockChunkManager_Mmap_Call { + _c.Call.Return(run) + return _c +} + +// MultiRead provides a mock function with given fields: ctx, filePaths +func (_m *MockChunkManager) MultiRead(ctx context.Context, filePaths []string) ([][]byte, error) { + ret := _m.Called(ctx, filePaths) + + if len(ret) == 0 { + panic("no return value specified for MultiRead") + } + + var r0 [][]byte + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, []string) ([][]byte, error)); ok { + return rf(ctx, filePaths) + } + if rf, ok := ret.Get(0).(func(context.Context, []string) [][]byte); ok { + r0 = rf(ctx, filePaths) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([][]byte) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, []string) error); ok { + r1 = rf(ctx, filePaths) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockChunkManager_MultiRead_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'MultiRead' +type MockChunkManager_MultiRead_Call struct { + *mock.Call +} + +// MultiRead is a helper method to define mock.On call +// - ctx context.Context +// - filePaths []string +func (_e *MockChunkManager_Expecter) MultiRead(ctx interface{}, filePaths interface{}) *MockChunkManager_MultiRead_Call { + return &MockChunkManager_MultiRead_Call{Call: _e.mock.On("MultiRead", ctx, filePaths)} +} + +func (_c *MockChunkManager_MultiRead_Call) Run(run func(ctx context.Context, filePaths []string)) *MockChunkManager_MultiRead_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].([]string)) + }) + return _c +} + +func (_c *MockChunkManager_MultiRead_Call) Return(_a0 [][]byte, _a1 error) *MockChunkManager_MultiRead_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockChunkManager_MultiRead_Call) RunAndReturn(run func(context.Context, []string) ([][]byte, error)) *MockChunkManager_MultiRead_Call { + _c.Call.Return(run) + return _c +} + +// MultiRemove provides a mock function with given fields: ctx, filePaths +func (_m *MockChunkManager) MultiRemove(ctx context.Context, filePaths []string) error { + ret := _m.Called(ctx, filePaths) + + if len(ret) == 0 { + panic("no return value specified for MultiRemove") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, []string) error); ok { + r0 = rf(ctx, filePaths) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockChunkManager_MultiRemove_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'MultiRemove' +type MockChunkManager_MultiRemove_Call struct { + *mock.Call +} + +// MultiRemove is a helper method to define mock.On call +// - ctx context.Context +// - filePaths []string +func (_e *MockChunkManager_Expecter) MultiRemove(ctx interface{}, filePaths interface{}) *MockChunkManager_MultiRemove_Call { + return &MockChunkManager_MultiRemove_Call{Call: _e.mock.On("MultiRemove", ctx, filePaths)} +} + +func (_c *MockChunkManager_MultiRemove_Call) Run(run func(ctx context.Context, filePaths []string)) *MockChunkManager_MultiRemove_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].([]string)) + }) + return _c +} + +func (_c *MockChunkManager_MultiRemove_Call) Return(_a0 error) *MockChunkManager_MultiRemove_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockChunkManager_MultiRemove_Call) RunAndReturn(run func(context.Context, []string) error) *MockChunkManager_MultiRemove_Call { + _c.Call.Return(run) + return _c +} + +// MultiWrite provides a mock function with given fields: ctx, contents +func (_m *MockChunkManager) MultiWrite(ctx context.Context, contents map[string][]byte) error { + ret := _m.Called(ctx, contents) + + if len(ret) == 0 { + panic("no return value specified for MultiWrite") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, map[string][]byte) error); ok { + r0 = rf(ctx, contents) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockChunkManager_MultiWrite_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'MultiWrite' +type MockChunkManager_MultiWrite_Call struct { + *mock.Call +} + +// MultiWrite is a helper method to define mock.On call +// - ctx context.Context +// - contents map[string][]byte +func (_e *MockChunkManager_Expecter) MultiWrite(ctx interface{}, contents interface{}) *MockChunkManager_MultiWrite_Call { + return &MockChunkManager_MultiWrite_Call{Call: _e.mock.On("MultiWrite", ctx, contents)} +} + +func (_c *MockChunkManager_MultiWrite_Call) Run(run func(ctx context.Context, contents map[string][]byte)) *MockChunkManager_MultiWrite_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(map[string][]byte)) + }) + return _c +} + +func (_c *MockChunkManager_MultiWrite_Call) Return(_a0 error) *MockChunkManager_MultiWrite_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockChunkManager_MultiWrite_Call) RunAndReturn(run func(context.Context, map[string][]byte) error) *MockChunkManager_MultiWrite_Call { + _c.Call.Return(run) + return _c +} + +// Path provides a mock function with given fields: ctx, filePath +func (_m *MockChunkManager) Path(ctx context.Context, filePath string) (string, error) { + ret := _m.Called(ctx, filePath) + + if len(ret) == 0 { + panic("no return value specified for Path") + } + + var r0 string + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, string) (string, error)); ok { + return rf(ctx, filePath) + } + if rf, ok := ret.Get(0).(func(context.Context, string) string); ok { + r0 = rf(ctx, filePath) + } else { + r0 = ret.Get(0).(string) + } + + if rf, ok := ret.Get(1).(func(context.Context, string) error); ok { + r1 = rf(ctx, filePath) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockChunkManager_Path_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Path' +type MockChunkManager_Path_Call struct { + *mock.Call +} + +// Path is a helper method to define mock.On call +// - ctx context.Context +// - filePath string +func (_e *MockChunkManager_Expecter) Path(ctx interface{}, filePath interface{}) *MockChunkManager_Path_Call { + return &MockChunkManager_Path_Call{Call: _e.mock.On("Path", ctx, filePath)} +} + +func (_c *MockChunkManager_Path_Call) Run(run func(ctx context.Context, filePath string)) *MockChunkManager_Path_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string)) + }) + return _c +} + +func (_c *MockChunkManager_Path_Call) Return(_a0 string, _a1 error) *MockChunkManager_Path_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockChunkManager_Path_Call) RunAndReturn(run func(context.Context, string) (string, error)) *MockChunkManager_Path_Call { + _c.Call.Return(run) + return _c +} + +// Read provides a mock function with given fields: ctx, filePath +func (_m *MockChunkManager) Read(ctx context.Context, filePath string) ([]byte, error) { + ret := _m.Called(ctx, filePath) + + if len(ret) == 0 { + panic("no return value specified for Read") + } + + var r0 []byte + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, string) ([]byte, error)); ok { + return rf(ctx, filePath) + } + if rf, ok := ret.Get(0).(func(context.Context, string) []byte); ok { + r0 = rf(ctx, filePath) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]byte) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, string) error); ok { + r1 = rf(ctx, filePath) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockChunkManager_Read_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Read' +type MockChunkManager_Read_Call struct { + *mock.Call +} + +// Read is a helper method to define mock.On call +// - ctx context.Context +// - filePath string +func (_e *MockChunkManager_Expecter) Read(ctx interface{}, filePath interface{}) *MockChunkManager_Read_Call { + return &MockChunkManager_Read_Call{Call: _e.mock.On("Read", ctx, filePath)} +} + +func (_c *MockChunkManager_Read_Call) Run(run func(ctx context.Context, filePath string)) *MockChunkManager_Read_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string)) + }) + return _c +} + +func (_c *MockChunkManager_Read_Call) Return(_a0 []byte, _a1 error) *MockChunkManager_Read_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockChunkManager_Read_Call) RunAndReturn(run func(context.Context, string) ([]byte, error)) *MockChunkManager_Read_Call { + _c.Call.Return(run) + return _c +} + +// ReadAt provides a mock function with given fields: ctx, filePath, off, length +func (_m *MockChunkManager) ReadAt(ctx context.Context, filePath string, off int64, length int64) ([]byte, error) { + ret := _m.Called(ctx, filePath, off, length) + + if len(ret) == 0 { + panic("no return value specified for ReadAt") + } + + var r0 []byte + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, string, int64, int64) ([]byte, error)); ok { + return rf(ctx, filePath, off, length) + } + if rf, ok := ret.Get(0).(func(context.Context, string, int64, int64) []byte); ok { + r0 = rf(ctx, filePath, off, length) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]byte) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, string, int64, int64) error); ok { + r1 = rf(ctx, filePath, off, length) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockChunkManager_ReadAt_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ReadAt' +type MockChunkManager_ReadAt_Call struct { + *mock.Call +} + +// ReadAt is a helper method to define mock.On call +// - ctx context.Context +// - filePath string +// - off int64 +// - length int64 +func (_e *MockChunkManager_Expecter) ReadAt(ctx interface{}, filePath interface{}, off interface{}, length interface{}) *MockChunkManager_ReadAt_Call { + return &MockChunkManager_ReadAt_Call{Call: _e.mock.On("ReadAt", ctx, filePath, off, length)} +} + +func (_c *MockChunkManager_ReadAt_Call) Run(run func(ctx context.Context, filePath string, off int64, length int64)) *MockChunkManager_ReadAt_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string), args[2].(int64), args[3].(int64)) + }) + return _c +} + +func (_c *MockChunkManager_ReadAt_Call) Return(p []byte, err error) *MockChunkManager_ReadAt_Call { + _c.Call.Return(p, err) + return _c +} + +func (_c *MockChunkManager_ReadAt_Call) RunAndReturn(run func(context.Context, string, int64, int64) ([]byte, error)) *MockChunkManager_ReadAt_Call { + _c.Call.Return(run) + return _c +} + +// Reader provides a mock function with given fields: ctx, filePath +func (_m *MockChunkManager) Reader(ctx context.Context, filePath string) (storage.FileReader, error) { + ret := _m.Called(ctx, filePath) + + if len(ret) == 0 { + panic("no return value specified for Reader") + } + + var r0 storage.FileReader + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, string) (storage.FileReader, error)); ok { + return rf(ctx, filePath) + } + if rf, ok := ret.Get(0).(func(context.Context, string) storage.FileReader); ok { + r0 = rf(ctx, filePath) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(storage.FileReader) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, string) error); ok { + r1 = rf(ctx, filePath) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockChunkManager_Reader_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Reader' +type MockChunkManager_Reader_Call struct { + *mock.Call +} + +// Reader is a helper method to define mock.On call +// - ctx context.Context +// - filePath string +func (_e *MockChunkManager_Expecter) Reader(ctx interface{}, filePath interface{}) *MockChunkManager_Reader_Call { + return &MockChunkManager_Reader_Call{Call: _e.mock.On("Reader", ctx, filePath)} +} + +func (_c *MockChunkManager_Reader_Call) Run(run func(ctx context.Context, filePath string)) *MockChunkManager_Reader_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string)) + }) + return _c +} + +func (_c *MockChunkManager_Reader_Call) Return(_a0 storage.FileReader, _a1 error) *MockChunkManager_Reader_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockChunkManager_Reader_Call) RunAndReturn(run func(context.Context, string) (storage.FileReader, error)) *MockChunkManager_Reader_Call { + _c.Call.Return(run) + return _c +} + +// Remove provides a mock function with given fields: ctx, filePath +func (_m *MockChunkManager) Remove(ctx context.Context, filePath string) error { + ret := _m.Called(ctx, filePath) + + if len(ret) == 0 { + panic("no return value specified for Remove") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, string) error); ok { + r0 = rf(ctx, filePath) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockChunkManager_Remove_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Remove' +type MockChunkManager_Remove_Call struct { + *mock.Call +} + +// Remove is a helper method to define mock.On call +// - ctx context.Context +// - filePath string +func (_e *MockChunkManager_Expecter) Remove(ctx interface{}, filePath interface{}) *MockChunkManager_Remove_Call { + return &MockChunkManager_Remove_Call{Call: _e.mock.On("Remove", ctx, filePath)} +} + +func (_c *MockChunkManager_Remove_Call) Run(run func(ctx context.Context, filePath string)) *MockChunkManager_Remove_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string)) + }) + return _c +} + +func (_c *MockChunkManager_Remove_Call) Return(_a0 error) *MockChunkManager_Remove_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockChunkManager_Remove_Call) RunAndReturn(run func(context.Context, string) error) *MockChunkManager_Remove_Call { + _c.Call.Return(run) + return _c +} + +// RemoveWithPrefix provides a mock function with given fields: ctx, prefix +func (_m *MockChunkManager) RemoveWithPrefix(ctx context.Context, prefix string) error { + ret := _m.Called(ctx, prefix) + + if len(ret) == 0 { + panic("no return value specified for RemoveWithPrefix") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, string) error); ok { + r0 = rf(ctx, prefix) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockChunkManager_RemoveWithPrefix_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RemoveWithPrefix' +type MockChunkManager_RemoveWithPrefix_Call struct { + *mock.Call +} + +// RemoveWithPrefix is a helper method to define mock.On call +// - ctx context.Context +// - prefix string +func (_e *MockChunkManager_Expecter) RemoveWithPrefix(ctx interface{}, prefix interface{}) *MockChunkManager_RemoveWithPrefix_Call { + return &MockChunkManager_RemoveWithPrefix_Call{Call: _e.mock.On("RemoveWithPrefix", ctx, prefix)} +} + +func (_c *MockChunkManager_RemoveWithPrefix_Call) Run(run func(ctx context.Context, prefix string)) *MockChunkManager_RemoveWithPrefix_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string)) + }) + return _c +} + +func (_c *MockChunkManager_RemoveWithPrefix_Call) Return(_a0 error) *MockChunkManager_RemoveWithPrefix_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockChunkManager_RemoveWithPrefix_Call) RunAndReturn(run func(context.Context, string) error) *MockChunkManager_RemoveWithPrefix_Call { + _c.Call.Return(run) + return _c +} + +// RootPath provides a mock function with given fields: +func (_m *MockChunkManager) RootPath() string { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for RootPath") + } + + var r0 string + if rf, ok := ret.Get(0).(func() string); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(string) + } + + return r0 +} + +// MockChunkManager_RootPath_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RootPath' +type MockChunkManager_RootPath_Call struct { + *mock.Call +} + +// RootPath is a helper method to define mock.On call +func (_e *MockChunkManager_Expecter) RootPath() *MockChunkManager_RootPath_Call { + return &MockChunkManager_RootPath_Call{Call: _e.mock.On("RootPath")} +} + +func (_c *MockChunkManager_RootPath_Call) Run(run func()) *MockChunkManager_RootPath_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockChunkManager_RootPath_Call) Return(_a0 string) *MockChunkManager_RootPath_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockChunkManager_RootPath_Call) RunAndReturn(run func() string) *MockChunkManager_RootPath_Call { + _c.Call.Return(run) + return _c +} + +// Size provides a mock function with given fields: ctx, filePath +func (_m *MockChunkManager) Size(ctx context.Context, filePath string) (int64, error) { + ret := _m.Called(ctx, filePath) + + if len(ret) == 0 { + panic("no return value specified for Size") + } + + var r0 int64 + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, string) (int64, error)); ok { + return rf(ctx, filePath) + } + if rf, ok := ret.Get(0).(func(context.Context, string) int64); ok { + r0 = rf(ctx, filePath) + } else { + r0 = ret.Get(0).(int64) + } + + if rf, ok := ret.Get(1).(func(context.Context, string) error); ok { + r1 = rf(ctx, filePath) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockChunkManager_Size_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Size' +type MockChunkManager_Size_Call struct { + *mock.Call +} + +// Size is a helper method to define mock.On call +// - ctx context.Context +// - filePath string +func (_e *MockChunkManager_Expecter) Size(ctx interface{}, filePath interface{}) *MockChunkManager_Size_Call { + return &MockChunkManager_Size_Call{Call: _e.mock.On("Size", ctx, filePath)} +} + +func (_c *MockChunkManager_Size_Call) Run(run func(ctx context.Context, filePath string)) *MockChunkManager_Size_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string)) + }) + return _c +} + +func (_c *MockChunkManager_Size_Call) Return(_a0 int64, _a1 error) *MockChunkManager_Size_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockChunkManager_Size_Call) RunAndReturn(run func(context.Context, string) (int64, error)) *MockChunkManager_Size_Call { + _c.Call.Return(run) + return _c +} + +// WalkWithPrefix provides a mock function with given fields: ctx, prefix, recursive, walkFunc +func (_m *MockChunkManager) WalkWithPrefix(ctx context.Context, prefix string, recursive bool, walkFunc storage.ChunkObjectWalkFunc) error { + ret := _m.Called(ctx, prefix, recursive, walkFunc) + + if len(ret) == 0 { + panic("no return value specified for WalkWithPrefix") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, string, bool, storage.ChunkObjectWalkFunc) error); ok { + r0 = rf(ctx, prefix, recursive, walkFunc) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockChunkManager_WalkWithPrefix_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'WalkWithPrefix' +type MockChunkManager_WalkWithPrefix_Call struct { + *mock.Call +} + +// WalkWithPrefix is a helper method to define mock.On call +// - ctx context.Context +// - prefix string +// - recursive bool +// - walkFunc storage.ChunkObjectWalkFunc +func (_e *MockChunkManager_Expecter) WalkWithPrefix(ctx interface{}, prefix interface{}, recursive interface{}, walkFunc interface{}) *MockChunkManager_WalkWithPrefix_Call { + return &MockChunkManager_WalkWithPrefix_Call{Call: _e.mock.On("WalkWithPrefix", ctx, prefix, recursive, walkFunc)} +} + +func (_c *MockChunkManager_WalkWithPrefix_Call) Run(run func(ctx context.Context, prefix string, recursive bool, walkFunc storage.ChunkObjectWalkFunc)) *MockChunkManager_WalkWithPrefix_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string), args[2].(bool), args[3].(storage.ChunkObjectWalkFunc)) + }) + return _c +} + +func (_c *MockChunkManager_WalkWithPrefix_Call) Return(_a0 error) *MockChunkManager_WalkWithPrefix_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockChunkManager_WalkWithPrefix_Call) RunAndReturn(run func(context.Context, string, bool, storage.ChunkObjectWalkFunc) error) *MockChunkManager_WalkWithPrefix_Call { + _c.Call.Return(run) + return _c +} + +// Write provides a mock function with given fields: ctx, filePath, content +func (_m *MockChunkManager) Write(ctx context.Context, filePath string, content []byte) error { + ret := _m.Called(ctx, filePath, content) + + if len(ret) == 0 { + panic("no return value specified for Write") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, string, []byte) error); ok { + r0 = rf(ctx, filePath, content) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockChunkManager_Write_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Write' +type MockChunkManager_Write_Call struct { + *mock.Call +} + +// Write is a helper method to define mock.On call +// - ctx context.Context +// - filePath string +// - content []byte +func (_e *MockChunkManager_Expecter) Write(ctx interface{}, filePath interface{}, content interface{}) *MockChunkManager_Write_Call { + return &MockChunkManager_Write_Call{Call: _e.mock.On("Write", ctx, filePath, content)} +} + +func (_c *MockChunkManager_Write_Call) Run(run func(ctx context.Context, filePath string, content []byte)) *MockChunkManager_Write_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string), args[2].([]byte)) + }) + return _c +} + +func (_c *MockChunkManager_Write_Call) Return(_a0 error) *MockChunkManager_Write_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockChunkManager_Write_Call) RunAndReturn(run func(context.Context, string, []byte) error) *MockChunkManager_Write_Call { + _c.Call.Return(run) + return _c +} + +// NewMockChunkManager creates a new instance of MockChunkManager. 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 NewMockChunkManager(t interface { + mock.TestingT + Cleanup(func()) +}) *MockChunkManager { + mock := &MockChunkManager{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/internal/mocks/streamingnode/client/handler/mock_consumer/mock_Consumer.go b/internal/mocks/streamingnode/client/handler/mock_consumer/mock_Consumer.go index efa7eb0f7f894..e9328568f4a89 100644 --- a/internal/mocks/streamingnode/client/handler/mock_consumer/mock_Consumer.go +++ b/internal/mocks/streamingnode/client/handler/mock_consumer/mock_Consumer.go @@ -18,8 +18,21 @@ func (_m *MockConsumer) EXPECT() *MockConsumer_Expecter { } // Close provides a mock function with given fields: -func (_m *MockConsumer) Close() { - _m.Called() +func (_m *MockConsumer) Close() error { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for Close") + } + + var r0 error + if rf, ok := ret.Get(0).(func() error); ok { + r0 = rf() + } else { + r0 = ret.Error(0) + } + + return r0 } // MockConsumer_Close_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Close' @@ -39,12 +52,12 @@ func (_c *MockConsumer_Close_Call) Run(run func()) *MockConsumer_Close_Call { return _c } -func (_c *MockConsumer_Close_Call) Return() *MockConsumer_Close_Call { - _c.Call.Return() +func (_c *MockConsumer_Close_Call) Return(_a0 error) *MockConsumer_Close_Call { + _c.Call.Return(_a0) return _c } -func (_c *MockConsumer_Close_Call) RunAndReturn(run func()) *MockConsumer_Close_Call { +func (_c *MockConsumer_Close_Call) RunAndReturn(run func() error) *MockConsumer_Close_Call { _c.Call.Return(run) return _c } diff --git a/internal/mocks/streamingnode/client/handler/mock_producer/mock_Producer.go b/internal/mocks/streamingnode/client/handler/mock_producer/mock_Producer.go index b215ccd60cc31..36d0a3714d24f 100644 --- a/internal/mocks/streamingnode/client/handler/mock_producer/mock_Producer.go +++ b/internal/mocks/streamingnode/client/handler/mock_producer/mock_Producer.go @@ -24,47 +24,61 @@ func (_m *MockProducer) EXPECT() *MockProducer_Expecter { return &MockProducer_Expecter{mock: &_m.Mock} } -// Assignment provides a mock function with given fields: -func (_m *MockProducer) Assignment() types.PChannelInfoAssigned { - ret := _m.Called() +// Append provides a mock function with given fields: ctx, msg +func (_m *MockProducer) Append(ctx context.Context, msg message.MutableMessage) (*types.AppendResult, error) { + ret := _m.Called(ctx, msg) if len(ret) == 0 { - panic("no return value specified for Assignment") + panic("no return value specified for Append") } - var r0 types.PChannelInfoAssigned - if rf, ok := ret.Get(0).(func() types.PChannelInfoAssigned); ok { - r0 = rf() + var r0 *types.AppendResult + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, message.MutableMessage) (*types.AppendResult, error)); ok { + return rf(ctx, msg) + } + if rf, ok := ret.Get(0).(func(context.Context, message.MutableMessage) *types.AppendResult); ok { + r0 = rf(ctx, msg) } else { - r0 = ret.Get(0).(types.PChannelInfoAssigned) + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.AppendResult) + } } - return r0 + if rf, ok := ret.Get(1).(func(context.Context, message.MutableMessage) error); ok { + r1 = rf(ctx, msg) + } else { + r1 = ret.Error(1) + } + + return r0, r1 } -// MockProducer_Assignment_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Assignment' -type MockProducer_Assignment_Call struct { +// MockProducer_Append_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Append' +type MockProducer_Append_Call struct { *mock.Call } -// Assignment is a helper method to define mock.On call -func (_e *MockProducer_Expecter) Assignment() *MockProducer_Assignment_Call { - return &MockProducer_Assignment_Call{Call: _e.mock.On("Assignment")} +// Append is a helper method to define mock.On call +// - ctx context.Context +// - msg message.MutableMessage +func (_e *MockProducer_Expecter) Append(ctx interface{}, msg interface{}) *MockProducer_Append_Call { + return &MockProducer_Append_Call{Call: _e.mock.On("Append", ctx, msg)} } -func (_c *MockProducer_Assignment_Call) Run(run func()) *MockProducer_Assignment_Call { +func (_c *MockProducer_Append_Call) Run(run func(ctx context.Context, msg message.MutableMessage)) *MockProducer_Append_Call { _c.Call.Run(func(args mock.Arguments) { - run() + run(args[0].(context.Context), args[1].(message.MutableMessage)) }) return _c } -func (_c *MockProducer_Assignment_Call) Return(_a0 types.PChannelInfoAssigned) *MockProducer_Assignment_Call { - _c.Call.Return(_a0) +func (_c *MockProducer_Append_Call) Return(_a0 *types.AppendResult, _a1 error) *MockProducer_Append_Call { + _c.Call.Return(_a0, _a1) return _c } -func (_c *MockProducer_Assignment_Call) RunAndReturn(run func() types.PChannelInfoAssigned) *MockProducer_Assignment_Call { +func (_c *MockProducer_Append_Call) RunAndReturn(run func(context.Context, message.MutableMessage) (*types.AppendResult, error)) *MockProducer_Append_Call { _c.Call.Return(run) return _c } @@ -193,65 +207,6 @@ func (_c *MockProducer_IsAvailable_Call) RunAndReturn(run func() bool) *MockProd return _c } -// Produce provides a mock function with given fields: ctx, msg -func (_m *MockProducer) Produce(ctx context.Context, msg message.MutableMessage) (*types.AppendResult, error) { - ret := _m.Called(ctx, msg) - - if len(ret) == 0 { - panic("no return value specified for Produce") - } - - var r0 *types.AppendResult - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, message.MutableMessage) (*types.AppendResult, error)); ok { - return rf(ctx, msg) - } - if rf, ok := ret.Get(0).(func(context.Context, message.MutableMessage) *types.AppendResult); ok { - r0 = rf(ctx, msg) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(*types.AppendResult) - } - } - - if rf, ok := ret.Get(1).(func(context.Context, message.MutableMessage) error); ok { - r1 = rf(ctx, msg) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// MockProducer_Produce_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Produce' -type MockProducer_Produce_Call struct { - *mock.Call -} - -// Produce is a helper method to define mock.On call -// - ctx context.Context -// - msg message.MutableMessage -func (_e *MockProducer_Expecter) Produce(ctx interface{}, msg interface{}) *MockProducer_Produce_Call { - return &MockProducer_Produce_Call{Call: _e.mock.On("Produce", ctx, msg)} -} - -func (_c *MockProducer_Produce_Call) Run(run func(ctx context.Context, msg message.MutableMessage)) *MockProducer_Produce_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(message.MutableMessage)) - }) - return _c -} - -func (_c *MockProducer_Produce_Call) Return(_a0 *types.AppendResult, _a1 error) *MockProducer_Produce_Call { - _c.Call.Return(_a0, _a1) - return _c -} - -func (_c *MockProducer_Produce_Call) RunAndReturn(run func(context.Context, message.MutableMessage) (*types.AppendResult, error)) *MockProducer_Produce_Call { - _c.Call.Return(run) - return _c -} - // NewMockProducer creates a new instance of MockProducer. 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 NewMockProducer(t interface { diff --git a/internal/mocks/streamingnode/server/mock_flusher/mock_Flusher.go b/internal/mocks/streamingnode/server/mock_flusher/mock_Flusher.go deleted file mode 100644 index a1ab70673be53..0000000000000 --- a/internal/mocks/streamingnode/server/mock_flusher/mock_Flusher.go +++ /dev/null @@ -1,246 +0,0 @@ -// Code generated by mockery v2.46.0. 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) - - if len(ret) == 0 { - panic("no return value specified for RegisterPChannel") - } - - var r0 error - if rf, ok := ret.Get(0).(func(string, wal.WAL) error); ok { - r0 = rf(pchannel, w) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// MockFlusher_RegisterPChannel_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RegisterPChannel' -type MockFlusher_RegisterPChannel_Call struct { - *mock.Call -} - -// RegisterPChannel is a helper method to define mock.On call -// - pchannel string -// - w wal.WAL -func (_e *MockFlusher_Expecter) RegisterPChannel(pchannel interface{}, w interface{}) *MockFlusher_RegisterPChannel_Call { - return &MockFlusher_RegisterPChannel_Call{Call: _e.mock.On("RegisterPChannel", pchannel, w)} -} - -func (_c *MockFlusher_RegisterPChannel_Call) Run(run func(pchannel string, w wal.WAL)) *MockFlusher_RegisterPChannel_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(string), args[1].(wal.WAL)) - }) - return _c -} - -func (_c *MockFlusher_RegisterPChannel_Call) Return(_a0 error) *MockFlusher_RegisterPChannel_Call { - _c.Call.Return(_a0) - return _c -} - -func (_c *MockFlusher_RegisterPChannel_Call) RunAndReturn(run func(string, wal.WAL) error) *MockFlusher_RegisterPChannel_Call { - _c.Call.Return(run) - return _c -} - -// RegisterVChannel provides a mock function with given fields: vchannel, _a1 -func (_m *MockFlusher) RegisterVChannel(vchannel string, _a1 wal.WAL) { - _m.Called(vchannel, _a1) -} - -// MockFlusher_RegisterVChannel_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RegisterVChannel' -type MockFlusher_RegisterVChannel_Call struct { - *mock.Call -} - -// RegisterVChannel is a helper method to define mock.On call -// - vchannel string -// - _a1 wal.WAL -func (_e *MockFlusher_Expecter) RegisterVChannel(vchannel interface{}, _a1 interface{}) *MockFlusher_RegisterVChannel_Call { - return &MockFlusher_RegisterVChannel_Call{Call: _e.mock.On("RegisterVChannel", vchannel, _a1)} -} - -func (_c *MockFlusher_RegisterVChannel_Call) Run(run func(vchannel string, _a1 wal.WAL)) *MockFlusher_RegisterVChannel_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(string), args[1].(wal.WAL)) - }) - return _c -} - -func (_c *MockFlusher_RegisterVChannel_Call) Return() *MockFlusher_RegisterVChannel_Call { - _c.Call.Return() - return _c -} - -func (_c *MockFlusher_RegisterVChannel_Call) RunAndReturn(run func(string, wal.WAL)) *MockFlusher_RegisterVChannel_Call { - _c.Call.Return(run) - return _c -} - -// Start provides a mock function with given fields: -func (_m *MockFlusher) Start() { - _m.Called() -} - -// MockFlusher_Start_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Start' -type MockFlusher_Start_Call struct { - *mock.Call -} - -// Start is a helper method to define mock.On call -func (_e *MockFlusher_Expecter) Start() *MockFlusher_Start_Call { - return &MockFlusher_Start_Call{Call: _e.mock.On("Start")} -} - -func (_c *MockFlusher_Start_Call) Run(run func()) *MockFlusher_Start_Call { - _c.Call.Run(func(args mock.Arguments) { - run() - }) - return _c -} - -func (_c *MockFlusher_Start_Call) Return() *MockFlusher_Start_Call { - _c.Call.Return() - return _c -} - -func (_c *MockFlusher_Start_Call) RunAndReturn(run func()) *MockFlusher_Start_Call { - _c.Call.Return(run) - return _c -} - -// Stop provides a mock function with given fields: -func (_m *MockFlusher) Stop() { - _m.Called() -} - -// MockFlusher_Stop_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Stop' -type MockFlusher_Stop_Call struct { - *mock.Call -} - -// Stop is a helper method to define mock.On call -func (_e *MockFlusher_Expecter) Stop() *MockFlusher_Stop_Call { - return &MockFlusher_Stop_Call{Call: _e.mock.On("Stop")} -} - -func (_c *MockFlusher_Stop_Call) Run(run func()) *MockFlusher_Stop_Call { - _c.Call.Run(func(args mock.Arguments) { - run() - }) - return _c -} - -func (_c *MockFlusher_Stop_Call) Return() *MockFlusher_Stop_Call { - _c.Call.Return() - return _c -} - -func (_c *MockFlusher_Stop_Call) RunAndReturn(run func()) *MockFlusher_Stop_Call { - _c.Call.Return(run) - return _c -} - -// UnregisterPChannel provides a mock function with given fields: pchannel -func (_m *MockFlusher) UnregisterPChannel(pchannel string) { - _m.Called(pchannel) -} - -// MockFlusher_UnregisterPChannel_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UnregisterPChannel' -type MockFlusher_UnregisterPChannel_Call struct { - *mock.Call -} - -// UnregisterPChannel is a helper method to define mock.On call -// - pchannel string -func (_e *MockFlusher_Expecter) UnregisterPChannel(pchannel interface{}) *MockFlusher_UnregisterPChannel_Call { - return &MockFlusher_UnregisterPChannel_Call{Call: _e.mock.On("UnregisterPChannel", pchannel)} -} - -func (_c *MockFlusher_UnregisterPChannel_Call) Run(run func(pchannel string)) *MockFlusher_UnregisterPChannel_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(string)) - }) - return _c -} - -func (_c *MockFlusher_UnregisterPChannel_Call) Return() *MockFlusher_UnregisterPChannel_Call { - _c.Call.Return() - return _c -} - -func (_c *MockFlusher_UnregisterPChannel_Call) RunAndReturn(run func(string)) *MockFlusher_UnregisterPChannel_Call { - _c.Call.Return(run) - return _c -} - -// UnregisterVChannel provides a mock function with given fields: vchannel -func (_m *MockFlusher) UnregisterVChannel(vchannel string) { - _m.Called(vchannel) -} - -// MockFlusher_UnregisterVChannel_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UnregisterVChannel' -type MockFlusher_UnregisterVChannel_Call struct { - *mock.Call -} - -// UnregisterVChannel is a helper method to define mock.On call -// - vchannel string -func (_e *MockFlusher_Expecter) UnregisterVChannel(vchannel interface{}) *MockFlusher_UnregisterVChannel_Call { - return &MockFlusher_UnregisterVChannel_Call{Call: _e.mock.On("UnregisterVChannel", vchannel)} -} - -func (_c *MockFlusher_UnregisterVChannel_Call) Run(run func(vchannel string)) *MockFlusher_UnregisterVChannel_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(string)) - }) - return _c -} - -func (_c *MockFlusher_UnregisterVChannel_Call) Return() *MockFlusher_UnregisterVChannel_Call { - _c.Call.Return() - return _c -} - -func (_c *MockFlusher_UnregisterVChannel_Call) RunAndReturn(run func(string)) *MockFlusher_UnregisterVChannel_Call { - _c.Call.Return(run) - return _c -} - -// NewMockFlusher creates a new instance of MockFlusher. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. -// The first argument is typically a *testing.T value. -func NewMockFlusher(t interface { - mock.TestingT - Cleanup(func()) -}) *MockFlusher { - mock := &MockFlusher{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} diff --git a/internal/mocks/streamingnode/server/wal/interceptors/mock_wab/mock_ROWriteAheadBuffer.go b/internal/mocks/streamingnode/server/wal/interceptors/mock_wab/mock_ROWriteAheadBuffer.go new file mode 100644 index 0000000000000..bad93162ad2a0 --- /dev/null +++ b/internal/mocks/streamingnode/server/wal/interceptors/mock_wab/mock_ROWriteAheadBuffer.go @@ -0,0 +1,96 @@ +// Code generated by mockery v2.46.0. DO NOT EDIT. + +package mock_wab + +import ( + context "context" + + wab "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/wab" + mock "github.com/stretchr/testify/mock" +) + +// MockROWriteAheadBuffer is an autogenerated mock type for the ROWriteAheadBuffer type +type MockROWriteAheadBuffer struct { + mock.Mock +} + +type MockROWriteAheadBuffer_Expecter struct { + mock *mock.Mock +} + +func (_m *MockROWriteAheadBuffer) EXPECT() *MockROWriteAheadBuffer_Expecter { + return &MockROWriteAheadBuffer_Expecter{mock: &_m.Mock} +} + +// ReadFromExclusiveTimeTick provides a mock function with given fields: ctx, timetick +func (_m *MockROWriteAheadBuffer) ReadFromExclusiveTimeTick(ctx context.Context, timetick uint64) (*wab.WriteAheadBufferReader, error) { + ret := _m.Called(ctx, timetick) + + if len(ret) == 0 { + panic("no return value specified for ReadFromExclusiveTimeTick") + } + + var r0 *wab.WriteAheadBufferReader + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, uint64) (*wab.WriteAheadBufferReader, error)); ok { + return rf(ctx, timetick) + } + if rf, ok := ret.Get(0).(func(context.Context, uint64) *wab.WriteAheadBufferReader); ok { + r0 = rf(ctx, timetick) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*wab.WriteAheadBufferReader) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, uint64) error); ok { + r1 = rf(ctx, timetick) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockROWriteAheadBuffer_ReadFromExclusiveTimeTick_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ReadFromExclusiveTimeTick' +type MockROWriteAheadBuffer_ReadFromExclusiveTimeTick_Call struct { + *mock.Call +} + +// ReadFromExclusiveTimeTick is a helper method to define mock.On call +// - ctx context.Context +// - timetick uint64 +func (_e *MockROWriteAheadBuffer_Expecter) ReadFromExclusiveTimeTick(ctx interface{}, timetick interface{}) *MockROWriteAheadBuffer_ReadFromExclusiveTimeTick_Call { + return &MockROWriteAheadBuffer_ReadFromExclusiveTimeTick_Call{Call: _e.mock.On("ReadFromExclusiveTimeTick", ctx, timetick)} +} + +func (_c *MockROWriteAheadBuffer_ReadFromExclusiveTimeTick_Call) Run(run func(ctx context.Context, timetick uint64)) *MockROWriteAheadBuffer_ReadFromExclusiveTimeTick_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(uint64)) + }) + return _c +} + +func (_c *MockROWriteAheadBuffer_ReadFromExclusiveTimeTick_Call) Return(_a0 *wab.WriteAheadBufferReader, _a1 error) *MockROWriteAheadBuffer_ReadFromExclusiveTimeTick_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockROWriteAheadBuffer_ReadFromExclusiveTimeTick_Call) RunAndReturn(run func(context.Context, uint64) (*wab.WriteAheadBufferReader, error)) *MockROWriteAheadBuffer_ReadFromExclusiveTimeTick_Call { + _c.Call.Return(run) + return _c +} + +// NewMockROWriteAheadBuffer creates a new instance of MockROWriteAheadBuffer. 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 NewMockROWriteAheadBuffer(t interface { + mock.TestingT + Cleanup(func()) +}) *MockROWriteAheadBuffer { + mock := &MockROWriteAheadBuffer{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/internal/mocks/streamingnode/server/wal/interceptors/timetick/mock_inspector/mock_TimeTickSyncOperator.go b/internal/mocks/streamingnode/server/wal/interceptors/timetick/mock_inspector/mock_TimeTickSyncOperator.go index b273f7da7a8d0..4b794eefece3f 100644 --- a/internal/mocks/streamingnode/server/wal/interceptors/timetick/mock_inspector/mock_TimeTickSyncOperator.go +++ b/internal/mocks/streamingnode/server/wal/interceptors/timetick/mock_inspector/mock_TimeTickSyncOperator.go @@ -5,10 +5,11 @@ package mock_inspector import ( context "context" - inspector "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick/inspector" mock "github.com/stretchr/testify/mock" types "github.com/milvus-io/milvus/pkg/streaming/util/types" + + wab "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/wab" ) // MockTimeTickSyncOperator is an autogenerated mock type for the TimeTickSyncOperator type @@ -102,49 +103,60 @@ func (_c *MockTimeTickSyncOperator_Sync_Call) RunAndReturn(run func(context.Cont return _c } -// TimeTickNotifier provides a mock function with given fields: -func (_m *MockTimeTickSyncOperator) TimeTickNotifier() *inspector.TimeTickNotifier { - ret := _m.Called() +// WriteAheadBuffer provides a mock function with given fields: ctx +func (_m *MockTimeTickSyncOperator) WriteAheadBuffer(ctx context.Context) (wab.ROWriteAheadBuffer, error) { + ret := _m.Called(ctx) if len(ret) == 0 { - panic("no return value specified for TimeTickNotifier") + panic("no return value specified for WriteAheadBuffer") } - var r0 *inspector.TimeTickNotifier - if rf, ok := ret.Get(0).(func() *inspector.TimeTickNotifier); ok { - r0 = rf() + var r0 wab.ROWriteAheadBuffer + var r1 error + if rf, ok := ret.Get(0).(func(context.Context) (wab.ROWriteAheadBuffer, error)); ok { + return rf(ctx) + } + if rf, ok := ret.Get(0).(func(context.Context) wab.ROWriteAheadBuffer); ok { + r0 = rf(ctx) } else { if ret.Get(0) != nil { - r0 = ret.Get(0).(*inspector.TimeTickNotifier) + r0 = ret.Get(0).(wab.ROWriteAheadBuffer) } } - return r0 + if rf, ok := ret.Get(1).(func(context.Context) error); ok { + r1 = rf(ctx) + } else { + r1 = ret.Error(1) + } + + return r0, r1 } -// MockTimeTickSyncOperator_TimeTickNotifier_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'TimeTickNotifier' -type MockTimeTickSyncOperator_TimeTickNotifier_Call struct { +// MockTimeTickSyncOperator_WriteAheadBuffer_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'WriteAheadBuffer' +type MockTimeTickSyncOperator_WriteAheadBuffer_Call struct { *mock.Call } -// TimeTickNotifier is a helper method to define mock.On call -func (_e *MockTimeTickSyncOperator_Expecter) TimeTickNotifier() *MockTimeTickSyncOperator_TimeTickNotifier_Call { - return &MockTimeTickSyncOperator_TimeTickNotifier_Call{Call: _e.mock.On("TimeTickNotifier")} +// WriteAheadBuffer is a helper method to define mock.On call +// - ctx context.Context +func (_e *MockTimeTickSyncOperator_Expecter) WriteAheadBuffer(ctx interface{}) *MockTimeTickSyncOperator_WriteAheadBuffer_Call { + return &MockTimeTickSyncOperator_WriteAheadBuffer_Call{Call: _e.mock.On("WriteAheadBuffer", ctx)} } -func (_c *MockTimeTickSyncOperator_TimeTickNotifier_Call) Run(run func()) *MockTimeTickSyncOperator_TimeTickNotifier_Call { +func (_c *MockTimeTickSyncOperator_WriteAheadBuffer_Call) Run(run func(ctx context.Context)) *MockTimeTickSyncOperator_WriteAheadBuffer_Call { _c.Call.Run(func(args mock.Arguments) { - run() + run(args[0].(context.Context)) }) return _c } -func (_c *MockTimeTickSyncOperator_TimeTickNotifier_Call) Return(_a0 *inspector.TimeTickNotifier) *MockTimeTickSyncOperator_TimeTickNotifier_Call { - _c.Call.Return(_a0) +func (_c *MockTimeTickSyncOperator_WriteAheadBuffer_Call) Return(_a0 wab.ROWriteAheadBuffer, _a1 error) *MockTimeTickSyncOperator_WriteAheadBuffer_Call { + _c.Call.Return(_a0, _a1) return _c } -func (_c *MockTimeTickSyncOperator_TimeTickNotifier_Call) RunAndReturn(run func() *inspector.TimeTickNotifier) *MockTimeTickSyncOperator_TimeTickNotifier_Call { +func (_c *MockTimeTickSyncOperator_WriteAheadBuffer_Call) RunAndReturn(run func(context.Context) (wab.ROWriteAheadBuffer, error)) *MockTimeTickSyncOperator_WriteAheadBuffer_Call { _c.Call.Return(run) return _c } diff --git a/internal/querycoordv2/balance/channel_level_score_balancer.go b/internal/querycoordv2/balance/channel_level_score_balancer.go index c17f3b25e7cfb..e16d8ad2b2d6b 100644 --- a/internal/querycoordv2/balance/channel_level_score_balancer.go +++ b/internal/querycoordv2/balance/channel_level_score_balancer.go @@ -93,6 +93,7 @@ func (b *ChannelLevelScoreBalancer) BalanceReplica(ctx context.Context, replica return b.ScoreBasedBalancer.BalanceReplica(ctx, replica) } + // TODO: assign by channel channelPlans = make([]ChannelAssignPlan, 0) segmentPlans = make([]SegmentAssignPlan, 0) for channelName := range channels { diff --git a/internal/streamingnode/client/handler/consumer/consumer.go b/internal/streamingnode/client/handler/consumer/consumer.go index d9fd1deb1abd1..32b0b3e995abe 100644 --- a/internal/streamingnode/client/handler/consumer/consumer.go +++ b/internal/streamingnode/client/handler/consumer/consumer.go @@ -14,5 +14,5 @@ type Consumer interface { Done() <-chan struct{} // Close the consumer, release the underlying resources. - Close() + Close() error } diff --git a/internal/streamingnode/client/handler/consumer/consumer_impl.go b/internal/streamingnode/client/handler/consumer/consumer_impl.go index fdd333d2364ea..c63b53ae36296 100644 --- a/internal/streamingnode/client/handler/consumer/consumer_impl.go +++ b/internal/streamingnode/client/handler/consumer/consumer_impl.go @@ -107,7 +107,7 @@ type consumerImpl struct { } // Close close the consumer client. -func (c *consumerImpl) Close() { +func (c *consumerImpl) Close() error { // Send the close request to server. if err := c.grpcStreamClient.Send(&streamingpb.ConsumeRequest{ Request: &streamingpb.ConsumeRequest_Close{}, @@ -118,7 +118,7 @@ func (c *consumerImpl) Close() { if err := c.grpcStreamClient.CloseSend(); err != nil { c.logger.Warn("close grpc stream failed", zap.Error(err)) } - <-c.finishErr.Done() + return c.finishErr.Get() } // Error returns the error of the consumer client. @@ -189,9 +189,12 @@ func (c *consumerImpl) recvLoop() (err error) { if c.txnBuilder != nil { panic("unreachable code: txn builder should be nil if we receive a non-txn message") } - if _, err := c.msgHandler.Handle(c.ctx, newImmutableMsg); err != nil { + if result := c.msgHandler.Handle(message.HandleParam{ + Ctx: c.ctx, + Message: newImmutableMsg, + }); result.Error != nil { c.logger.Warn("message handle canceled", zap.Error(err)) - return errors.Wrapf(err, "At Handler") + return errors.Wrapf(result.Error, "At Handler") } } case *streamingpb.ConsumeResponse_Close: @@ -255,7 +258,10 @@ func (c *consumerImpl) handleTxnMessage(msg message.ImmutableMessage) error { c.logger.Warn("failed to build txn message", zap.Any("messageID", commitMsg.MessageID()), zap.Error(err)) return nil } - if _, err := c.msgHandler.Handle(c.ctx, msg); err != nil { + if result := c.msgHandler.Handle(message.HandleParam{ + Ctx: c.ctx, + Message: msg, + }); result.Error != nil { c.logger.Warn("message handle canceled at txn", zap.Error(err)) return errors.Wrap(err, "At Handler Of Txn") } diff --git a/internal/streamingnode/client/handler/consumer/consumer_test.go b/internal/streamingnode/client/handler/consumer/consumer_test.go index a0bebccf96db3..97dfca0e338d0 100644 --- a/internal/streamingnode/client/handler/consumer/consumer_test.go +++ b/internal/streamingnode/client/handler/consumer/consumer_test.go @@ -14,6 +14,7 @@ import ( "github.com/milvus-io/milvus/pkg/proto/messagespb" "github.com/milvus-io/milvus/pkg/proto/streamingpb" "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/streaming/util/types" "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" @@ -21,7 +22,7 @@ import ( ) func TestConsumer(t *testing.T) { - resultCh := make(message.ChanMessageHandler, 1) + resultCh := make(adaptor.ChanMessageHandler, 1) c := newMockedConsumerImpl(t, context.Background(), resultCh) mmsg, _ := message.NewInsertMessageBuilderV1(). @@ -70,7 +71,7 @@ func TestConsumer(t *testing.T) { } func TestConsumerWithCancellation(t *testing.T) { - resultCh := make(message.ChanMessageHandler, 1) + resultCh := make(adaptor.ChanMessageHandler, 1) ctx, cancel := context.WithCancel(context.Background()) c := newMockedConsumerImpl(t, ctx, resultCh) diff --git a/internal/streamingnode/client/handler/handler_client_impl.go b/internal/streamingnode/client/handler/handler_client_impl.go index e8266af323a59..7a11133c11944 100644 --- a/internal/streamingnode/client/handler/handler_client_impl.go +++ b/internal/streamingnode/client/handler/handler_client_impl.go @@ -11,6 +11,8 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/client/handler/assignment" "github.com/milvus-io/milvus/internal/streamingnode/client/handler/consumer" "github.com/milvus-io/milvus/internal/streamingnode/client/handler/producer" + "github.com/milvus-io/milvus/internal/streamingnode/client/handler/registry" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/util/streamingutil/service/balancer/picker" "github.com/milvus-io/milvus/internal/util/streamingutil/service/lazygrpc" "github.com/milvus-io/milvus/internal/util/streamingutil/service/resolver" @@ -21,7 +23,11 @@ import ( "github.com/milvus-io/milvus/pkg/util/typeutil" ) -var errWaitNextBackoff = errors.New("wait for next backoff") +var ( + errWaitNextBackoff = errors.New("wait for next backoff") + _ producer.Producer = wal.WAL(nil) + _ consumer.Consumer = wal.Scanner(nil) +) type handlerClientImpl struct { lifetime *typeutil.Lifetime @@ -40,15 +46,27 @@ func (hc *handlerClientImpl) CreateProducer(ctx context.Context, opts *ProducerO } defer hc.lifetime.Done() - p, err := hc.createHandlerAfterStreamingNodeReady(ctx, opts.PChannel, func(ctx context.Context, assign *types.PChannelInfoAssigned) (any, error) { + p, err := hc.createHandlerAfterStreamingNodeReady(ctx, opts.PChannel, func(ctx context.Context, assign *types.PChannelInfoAssigned) (*handlerCreateResult, error) { + // Check if the localWAL is assigned at local + localWAL, err := registry.GetAvailableWAL(assign.Channel) + if err == nil { + return localResult(localWAL), nil + } + if !shouldUseRemoteWAL(err) { + return nil, err + } // Wait for handler service is ready. handlerService, err := hc.service.GetService(ctx) if err != nil { return nil, err } - return hc.newProducer(ctx, &producer.ProducerOptions{ + remoteWAL, err := hc.newProducer(ctx, &producer.ProducerOptions{ Assignment: assign, }, handlerService) + if err != nil { + return nil, err + } + return remoteResult(remoteWAL), nil }) if err != nil { return nil, err @@ -63,19 +81,41 @@ func (hc *handlerClientImpl) CreateConsumer(ctx context.Context, opts *ConsumerO } defer hc.lifetime.Done() - c, err := hc.createHandlerAfterStreamingNodeReady(ctx, opts.PChannel, func(ctx context.Context, assign *types.PChannelInfoAssigned) (any, error) { + c, err := hc.createHandlerAfterStreamingNodeReady(ctx, opts.PChannel, func(ctx context.Context, assign *types.PChannelInfoAssigned) (*handlerCreateResult, error) { + // Check if the localWAL is assigned at local + localWAL, err := registry.GetAvailableWAL(assign.Channel) + if err == nil { + localScanner, err := localWAL.Read(ctx, wal.ReadOption{ + VChannel: opts.VChannel, + DeliverPolicy: opts.DeliverPolicy, + MessageFilter: opts.DeliverFilters, + MesasgeHandler: opts.MessageHandler, + }) + if err != nil { + return nil, err + } + return localResult(localScanner), nil + } + if !shouldUseRemoteWAL(err) { + return nil, err + } + // Wait for handler service is ready. handlerService, err := hc.service.GetService(ctx) if err != nil { return nil, err } - return hc.newConsumer(ctx, &consumer.ConsumerOptions{ + remoteScanner, err := hc.newConsumer(ctx, &consumer.ConsumerOptions{ Assignment: assign, VChannel: opts.VChannel, DeliverPolicy: opts.DeliverPolicy, DeliverFilters: opts.DeliverFilters, MessageHandler: opts.MessageHandler, }, handlerService) + if err != nil { + return nil, err + } + return remoteResult(remoteScanner), nil }) if err != nil { return nil, err @@ -83,9 +123,24 @@ func (hc *handlerClientImpl) CreateConsumer(ctx context.Context, opts *ConsumerO return c.(Consumer), nil } +func localResult(result any) *handlerCreateResult { + return &handlerCreateResult{result: result, isLocal: true} +} + +func remoteResult(result any) *handlerCreateResult { + return &handlerCreateResult{result: result, isLocal: false} +} + +type handlerCreateResult struct { + result any + isLocal bool +} + +type handlerCreateFunc func(ctx context.Context, assign *types.PChannelInfoAssigned) (*handlerCreateResult, error) + // createHandlerAfterStreamingNodeReady creates a handler until streaming node ready. // If streaming node is not ready, it will block until new assignment term is coming or context timeout. -func (hc *handlerClientImpl) createHandlerAfterStreamingNodeReady(ctx context.Context, pchannel string, create func(ctx context.Context, assign *types.PChannelInfoAssigned) (any, error)) (any, error) { +func (hc *handlerClientImpl) createHandlerAfterStreamingNodeReady(ctx context.Context, pchannel string, create handlerCreateFunc) (any, error) { logger := log.With(zap.String("pchannel", pchannel)) // TODO: backoff should be configurable. backoff := backoff.NewExponentialBackOff() @@ -93,9 +148,10 @@ func (hc *handlerClientImpl) createHandlerAfterStreamingNodeReady(ctx context.Co assign := hc.watcher.Get(ctx, pchannel) if assign != nil { // Find assignment, try to create producer on this assignment. - c, err := create(ctx, assign) + createResult, err := create(ctx, assign) if err == nil { - return c, nil + logger.Info("create handler success", zap.Any("assignment", assign), zap.Bool("isLocal", createResult.isLocal)) + return createResult.result, nil } logger.Warn("create handler failed", zap.Any("assignment", assign), zap.Error(err)) @@ -158,3 +214,18 @@ func isPermanentFailureUntilNewAssignment(err error) bool { streamingServiceErr := status.AsStreamingError(err) return streamingServiceErr.IsWrongStreamingNode() } + +// shouldUseRemoteWAL checks if use remote wal when given error happens. +func shouldUseRemoteWAL(err error) bool { + if err == nil { + panic("the incoming error should never be nil") + } + // When following error happens, we should try to make a remote wal fetch. + // 1. If current node didn't deploy any streaming node. + if errors.Is(err, registry.ErrNoStreamingNodeDeployed) { + return true + } + // 2. If the wal is not exist at current streaming node. + streamingServiceErr := status.AsStreamingError(err) + return streamingServiceErr.IsWrongStreamingNode() +} diff --git a/internal/streamingnode/client/handler/handler_client_test.go b/internal/streamingnode/client/handler/handler_client_test.go index 584a34e135632..f848462b16b49 100644 --- a/internal/streamingnode/client/handler/handler_client_test.go +++ b/internal/streamingnode/client/handler/handler_client_test.go @@ -19,7 +19,7 @@ import ( "github.com/milvus-io/milvus/pkg/mocks/proto/mock_streamingpb" "github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_types" "github.com/milvus-io/milvus/pkg/proto/streamingpb" - "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/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/paramtable" @@ -41,9 +41,9 @@ func TestHandlerClient(t *testing.T) { w.EXPECT().Close().Run(func() {}) p := mock_producer.NewMockProducer(t) - p.EXPECT().Close().Run(func() {}) + p.EXPECT().Close().RunAndReturn(func() {}) c := mock_consumer.NewMockConsumer(t) - c.EXPECT().Close().Run(func() {}) + c.EXPECT().Close().RunAndReturn(func() error { return nil }) rebalanceTrigger := mock_types.NewMockAssignmentRebalanceTrigger(t) rebalanceTrigger.EXPECT().ReportAssignmentError(mock.Anything, mock.Anything, mock.Anything).Return(nil) @@ -104,7 +104,7 @@ func TestHandlerClient(t *testing.T) { options.DeliverFilterTimeTickGT(10), options.DeliverFilterTimeTickGTE(10), }, - MessageHandler: make(message.ChanMessageHandler), + MessageHandler: make(adaptor.ChanMessageHandler), }) assert.NoError(t, err) assert.NotNil(t, consumer) diff --git a/internal/streamingnode/client/handler/producer/producer.go b/internal/streamingnode/client/handler/producer/producer.go index 41dec673d9d6c..6e31d2b479010 100644 --- a/internal/streamingnode/client/handler/producer/producer.go +++ b/internal/streamingnode/client/handler/producer/producer.go @@ -9,18 +9,13 @@ import ( var _ Producer = (*producerImpl)(nil) -type ProduceResult = types.AppendResult - // Producer is the interface that wraps the basic produce method on grpc stream. // Producer is work on a single stream on grpc, // so Producer cannot recover from failure because of the stream is broken. type Producer interface { - // Assignment returns the assignment of the producer. - Assignment() types.PChannelInfoAssigned - - // Produce sends the produce message to server. + // Append sends the produce message to server. // TODO: Support Batch produce here. - Produce(ctx context.Context, msg message.MutableMessage) (*ProduceResult, error) + Append(ctx context.Context, msg message.MutableMessage) (*types.AppendResult, error) // Check if a producer is available. IsAvailable() bool diff --git a/internal/streamingnode/client/handler/producer/producer_impl.go b/internal/streamingnode/client/handler/producer/producer_impl.go index 9c84df18429c0..5ed601ef92f4a 100644 --- a/internal/streamingnode/client/handler/producer/producer_impl.go +++ b/internal/streamingnode/client/handler/producer/producer_impl.go @@ -114,17 +114,12 @@ type produceRequest struct { } type produceResponse struct { - result *ProduceResult + result *types.AppendResult err error } -// Assignment returns the assignment of the producer. -func (p *producerImpl) Assignment() types.PChannelInfoAssigned { - return p.assignment -} - -// Produce sends the produce message to server. -func (p *producerImpl) Produce(ctx context.Context, msg message.MutableMessage) (*ProduceResult, error) { +// Append sends the produce message to server. +func (p *producerImpl) Append(ctx context.Context, msg message.MutableMessage) (*types.AppendResult, error) { if !p.lifetime.Add(typeutil.LifetimeStateWorking) { return nil, status.NewOnShutdownError("producer client is shutting down") } @@ -293,7 +288,7 @@ func (p *producerImpl) recvLoop() (err error) { return err } result = produceResponse{ - result: &ProduceResult{ + result: &types.AppendResult{ MessageID: msgID, TimeTick: produceResp.Result.GetTimetick(), TxnCtx: message.NewTxnContextFromProto(produceResp.Result.GetTxnContext()), diff --git a/internal/streamingnode/client/handler/producer/producer_test.go b/internal/streamingnode/client/handler/producer/producer_test.go index c65c8d963ee1c..ab49418100960 100644 --- a/internal/streamingnode/client/handler/producer/producer_test.go +++ b/internal/streamingnode/client/handler/producer/producer_test.go @@ -61,12 +61,12 @@ func TestProducer(t *testing.T) { ch := make(chan struct{}) go func() { msg := message.CreateTestEmptyInsertMesage(1, nil) - msgID, err := producer.Produce(ctx, msg) + msgID, err := producer.Append(ctx, msg) assert.Error(t, err) assert.Nil(t, msgID) msg = message.CreateTestEmptyInsertMesage(1, nil) - msgID, err = producer.Produce(ctx, msg) + msgID, err = producer.Append(ctx, msg) assert.NoError(t, err) assert.NotNil(t, msgID) close(ch) @@ -100,7 +100,7 @@ func TestProducer(t *testing.T) { ctx, cancel := context.WithTimeout(ctx, 100*time.Millisecond) defer cancel() msg := message.CreateTestEmptyInsertMesage(1, nil) - _, err = producer.Produce(ctx, msg) + _, err = producer.Append(ctx, msg) assert.ErrorIs(t, err, context.DeadlineExceeded) assert.True(t, producer.IsAvailable()) producer.Close() diff --git a/internal/streamingnode/client/handler/registry/wal_manager.go b/internal/streamingnode/client/handler/registry/wal_manager.go new file mode 100644 index 0000000000000..36249e1616902 --- /dev/null +++ b/internal/streamingnode/client/handler/registry/wal_manager.go @@ -0,0 +1,44 @@ +package registry + +import ( + "context" + + "github.com/cockroachdb/errors" + + "github.com/milvus-io/milvus/internal/streamingnode/server/wal" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/streaming/util/types" + "github.com/milvus-io/milvus/pkg/util/paramtable" + "github.com/milvus-io/milvus/pkg/util/syncutil" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +var ( + registry = syncutil.NewFuture[WALManager]() + ErrNoStreamingNodeDeployed = errors.New("no streaming node deployed") +) + +// RegisterLocalWALManager registers the local wal manager. +// When the streaming node is started, it should call this function to register the wal manager. +func RegisterLocalWALManager(manager WALManager) { + if !paramtable.IsLocalComponentEnabled(typeutil.StreamingNodeRole) { + panic("unreachable: streaming node is not enabled but wal setup") + } + registry.Set(manager) + log.Ctx(context.Background()).Info("register local wal manager done") +} + +// GetAvailableWAL returns a available wal instance for the channel. +func GetAvailableWAL(channel types.PChannelInfo) (wal.WAL, error) { + if !paramtable.IsLocalComponentEnabled(typeutil.StreamingNodeRole) { + return nil, ErrNoStreamingNodeDeployed + } + return registry.Get().GetAvailableWAL(channel) +} + +// WALManager is a hint type for wal manager at streaming node. +type WALManager interface { + // GetAvailableWAL returns a available wal instance for the channel. + // Return nil if the wal instance is not found. + GetAvailableWAL(channel types.PChannelInfo) (wal.WAL, error) +} diff --git a/internal/streamingnode/server/builder.go b/internal/streamingnode/server/builder.go index f35e76b233375..a645c083153c9 100644 --- a/internal/streamingnode/server/builder.go +++ b/internal/streamingnode/server/builder.go @@ -6,7 +6,6 @@ import ( "github.com/milvus-io/milvus/internal/metastore/kv/streamingnode" "github.com/milvus-io/milvus/internal/storage" - "github.com/milvus-io/milvus/internal/streamingnode/server/flusher/flusherimpl" "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/sessionutil" @@ -77,13 +76,11 @@ func (b *ServerBuilder) WithMetaKV(kv kv.MetaKv) *ServerBuilder { func (b *ServerBuilder) Build() *Server { resource.Apply( resource.OptETCD(b.etcdClient), + resource.OptChunkManager(b.chunkManager), resource.OptRootCoordClient(b.rc), resource.OptDataCoordClient(b.dc), resource.OptStreamingNodeCatalog(streamingnode.NewCataLog(b.kv)), ) - resource.Apply( - resource.OptFlusher(flusherimpl.NewFlusher(b.chunkManager)), - ) resource.Done() return &Server{ session: b.session, diff --git a/internal/streamingnode/server/flusher/flusher.go b/internal/streamingnode/server/flusher/flusher.go deleted file mode 100644 index 594d5b3bc2944..0000000000000 --- a/internal/streamingnode/server/flusher/flusher.go +++ /dev/null @@ -1,40 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package flusher - -import "github.com/milvus-io/milvus/internal/streamingnode/server/wal" - -type Flusher interface { - // RegisterPChannel ASYNCHRONOUSLY creates and starts pipelines belonging to the pchannel/WAL. - // If a pipeline creation fails, the flusher will keep retrying to create it indefinitely. - RegisterPChannel(pchannel string, w wal.WAL) error - - // UnregisterPChannel stops and removes pipelines belonging to the pchannel. - UnregisterPChannel(pchannel string) - - // RegisterVChannel ASYNCHRONOUSLY create pipeline belonging to the vchannel. - RegisterVChannel(vchannel string, wal wal.WAL) - - // UnregisterVChannel stops and removes pipeline belonging to the vchannel. - UnregisterVChannel(vchannel string) - - // Start flusher service. - Start() - - // Stop flusher, will synchronously flush all remaining data. - Stop() -} diff --git a/internal/streamingnode/server/flusher/flusherimpl/channel_lifetime.go b/internal/streamingnode/server/flusher/flusherimpl/channel_lifetime.go deleted file mode 100644 index fb62c511565de..0000000000000 --- a/internal/streamingnode/server/flusher/flusherimpl/channel_lifetime.go +++ /dev/null @@ -1,173 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package flusherimpl - -import ( - "context" - "math" - "sync" - "time" - - "github.com/cockroachdb/errors" - "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/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/internal/streamingnode/server/wal/interceptors/segment/stats" - "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/proto/datapb" - "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/merr" -) - -type LifetimeState int - -const ( - Pending LifetimeState = iota - Cancel - Fail - Done -) - -var errChannelLifetimeUnrecoverable = errors.New("channel lifetime unrecoverable") - -type ChannelLifetime interface { - Run() error - Cancel() -} - -type channelLifetime struct { - mu sync.Mutex - state LifetimeState - vchannel string - wal wal.WAL - scanner wal.Scanner - f *flusherImpl -} - -func NewChannelLifetime(f *flusherImpl, vchannel string, wal wal.WAL) ChannelLifetime { - return &channelLifetime{ - state: Pending, - f: f, - vchannel: vchannel, - wal: wal, - } -} - -func (c *channelLifetime) Run() error { - c.mu.Lock() - defer c.mu.Unlock() - if c.state == Cancel || c.state == Done { - return nil - } - if c.state == Fail { - return errChannelLifetimeUnrecoverable - } - log.Info("start to build pipeline", zap.String("vchannel", c.vchannel)) - - // Get recovery info from datacoord. - ctx, cancel := context.WithTimeout(context.Background(), time.Second*30) - defer cancel() - - pipelineParams, err := c.f.getPipelineParams(ctx) - if err != nil { - return err - } - - dc, err := resource.Resource().DataCoordClient().GetWithContext(ctx) - if err != nil { - return errors.Wrap(err, "At Get DataCoordClient") - } - resp, err := dc.GetChannelRecoveryInfo(ctx, &datapb.GetChannelRecoveryInfoRequest{Vchannel: c.vchannel}) - if err = merr.CheckRPCCall(resp, err); err != nil { - return err - } - // The channel has been dropped, skip to recover it. - if len(resp.GetInfo().GetSeekPosition().GetMsgID()) == 0 && resp.GetInfo().GetSeekPosition().GetTimestamp() == math.MaxUint64 { - log.Info("channel has been dropped, skip to create flusher for vchannel", zap.String("vchannel", c.vchannel)) - c.state = Fail - return errChannelLifetimeUnrecoverable - } - - // Convert common.MessageID to message.messageID. - messageID := adaptor.MustGetMessageIDFromMQWrapperIDBytes(c.wal.WALName(), resp.GetInfo().GetSeekPosition().GetMsgID()) - - // Create scanner. - policy := options.DeliverPolicyStartFrom(messageID) - handler := adaptor2.NewMsgPackAdaptorHandler() - ro := wal.ReadOption{ - VChannel: c.vchannel, - DeliverPolicy: policy, - MesasgeHandler: handler, - } - scanner, err := c.wal.Read(ctx, ro) - if err != nil { - return err - } - - // Build and add pipeline. - ds, err := pipeline.NewStreamingNodeDataSyncService(ctx, pipelineParams, - // TODO fubang add the db properties - &datapb.ChannelWatchInfo{Vchan: resp.GetInfo(), Schema: resp.GetSchema()}, handler.Chan(), func(t syncmgr.Task, err error) { - if err != nil || t == nil { - return - } - if tt, ok := t.(*syncmgr.SyncTask); ok { - insertLogs, _, _, _ := tt.Binlogs() - resource.Resource().SegmentAssignStatsManager().UpdateOnSync(tt.SegmentID(), stats.SyncOperationMetrics{ - BinLogCounterIncr: 1, - BinLogFileCounterIncr: uint64(len(insertLogs)), - }) - } - }, - func() { go func() { c.Cancel() }() }, - ) - if err != nil { - scanner.Close() - return err - } - ds.Start() - c.f.fgMgr.AddFlowgraph(ds) - c.scanner = scanner - c.state = Done - - log.Info("build pipeline done", zap.String("vchannel", c.vchannel)) - return nil -} - -func (c *channelLifetime) Cancel() { - c.mu.Lock() - defer c.mu.Unlock() - switch c.state { - case Pending: - c.state = Cancel - case Cancel, Fail: - return - case Done: - err := c.scanner.Close() - if err != nil { - log.Warn("scanner error", zap.String("vchannel", c.vchannel), zap.Error(err)) - } - c.f.fgMgr.RemoveFlowgraph(c.vchannel) - c.f.wbMgr.RemoveChannel(c.vchannel) - log.Info("flusher unregister vchannel done", zap.String("vchannel", c.vchannel)) - } -} diff --git a/internal/streamingnode/server/flusher/flusherimpl/flusher_components.go b/internal/streamingnode/server/flusher/flusherimpl/flusher_components.go new file mode 100644 index 0000000000000..a26804f31030e --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/flusher_components.go @@ -0,0 +1,329 @@ +package flusherimpl + +import ( + "context" + + "go.uber.org/zap" + "google.golang.org/protobuf/proto" + + "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/broker" + "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/storage" + "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/segment/stats" + "github.com/milvus-io/milvus/internal/util/idalloc" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "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/util/conc" + "github.com/milvus-io/milvus/pkg/util/paramtable" + "github.com/milvus-io/milvus/pkg/util/retry" +) + +// buildFlusherComponents builds the components of the flusher. +func (impl *WALFlusherImpl) buildFlusherComponents(ctx context.Context, l wal.WAL) (*flusherComponents, error) { + // Get all existed vchannels of the pchannel. + vchannels, err := impl.getVchannels(ctx, l.Channel().Name) + if err != nil { + impl.logger.Warn("get vchannels failed", zap.Error(err)) + return nil, err + } + impl.logger.Info("fetch vchannel done", zap.Int("vchannelNum", len(vchannels))) + + // Get all the recovery info of the recoverable vchannels. + recoverInfos, err := impl.getRecoveryInfos(ctx, vchannels) + if err != nil { + impl.logger.Warn("get recovery info failed", zap.Error(err)) + return nil, err + } + impl.logger.Info("fetch recovery info done", zap.Int("recoveryInfoNum", len(recoverInfos))) + + // build up components + dc, err := resource.Resource().DataCoordClient().GetWithContext(ctx) + if err != nil { + impl.logger.Warn("flusher recovery is canceled before data coord client ready", zap.Error(err)) + return nil, err + } + impl.logger.Info("data coord client ready") + + broker := broker.NewCoordBroker(dc, paramtable.GetNodeID()) + chunkManager := resource.Resource().ChunkManager() + syncMgr := syncmgr.NewSyncManager(chunkManager) + wbMgr := writebuffer.NewManager(syncMgr) + wbMgr.Start() + cpUpdater := util.NewChannelCheckpointUpdater(broker) + go cpUpdater.Start() + + fc := &flusherComponents{ + wal: l, + broker: broker, + fgMgr: pipeline.NewFlowgraphManager(), + syncMgr: syncMgr, + wbMgr: wbMgr, + cpUpdater: cpUpdater, + chunkManager: chunkManager, + dataServices: make(map[string]*dataService), + logger: impl.logger, + } + impl.logger.Info("flusher components intiailizing done") + if err := fc.recover(ctx, recoverInfos); err != nil { + impl.logger.Warn("flusher recovery is canceled before recovery done, recycle the resource", zap.Error(err)) + fc.Close() + impl.logger.Info("flusher recycle the resource done") + return nil, err + } + impl.logger.Info("flusher recovery done") + return fc, nil +} + +type dataService struct { + input chan<- *msgstream.MsgPack + handler *adaptor.BaseMsgPackAdaptorHandler + ds *pipeline.DataSyncService + startMessageID message.MessageID +} + +func (ds *dataService) HandleMessage(ctx context.Context, msg message.ImmutableMessage) error { + ds.handler.GenerateMsgPack(msg) + for ds.handler.PendingMsgPack.Len() > 0 { + select { + case <-ctx.Done(): + return ctx.Err() + case ds.input <- ds.handler.PendingMsgPack.Next(): + ds.handler.PendingMsgPack.UnsafeAdvance() + } + } + return nil +} + +func (ds *dataService) Close() { + close(ds.input) + ds.ds.GracefullyClose() +} + +// flusherComponents is the components of the flusher. +type flusherComponents struct { + wal wal.WAL + broker broker.Broker + fgMgr pipeline.FlowgraphManager + syncMgr syncmgr.SyncManager + wbMgr writebuffer.BufferManager + cpUpdater *util.ChannelCheckpointUpdater + chunkManager storage.ChunkManager + dataServices map[string]*dataService + logger *log.MLogger +} + +// GetMinimumStartMessage gets the minimum start message of all the data services. +func (impl *flusherComponents) GetMinimumStartMessage() message.MessageID { + var startMessageID message.MessageID + for _, ds := range impl.dataServices { + if startMessageID == nil || ds.startMessageID.LT(startMessageID) { + startMessageID = ds.startMessageID + } + } + return startMessageID +} + +func (impl *flusherComponents) WhenCreateCollection(createCollectionMsg message.ImmutableCreateCollectionMessageV1) { + if _, ok := impl.dataServices[createCollectionMsg.VChannel()]; ok { + impl.logger.Info("the data sync service of current vchannel is built, skip it", zap.String("vchannel", createCollectionMsg.VChannel())) + // May repeated consumed, so we ignore the message. + return + } + createCollectionRequest, err := createCollectionMsg.Body() + if err != nil { + panic("the message body is not CreateCollectionRequest") + } + msgChan := make(chan *msgstream.MsgPack, 10) + + schema := &schemapb.CollectionSchema{} + if err := proto.Unmarshal(createCollectionRequest.GetSchema(), schema); err != nil { + panic("failed to unmarshal collection schema") + } + ds := pipeline.NewEmptyStreamingNodeDataSyncService( + context.Background(), // There's no any rpc in this function, so the context is not used here. + &util.PipelineParams{ + Ctx: context.Background(), + Broker: impl.broker, + SyncMgr: impl.syncMgr, + ChunkManager: impl.chunkManager, + WriteBufferManager: impl.wbMgr, + CheckpointUpdater: impl.cpUpdater, + Allocator: idalloc.NewMAllocator(resource.Resource().IDAllocator()), + MsgHandler: newMsgHandler(impl.wbMgr), + }, + msgChan, + &datapb.VchannelInfo{ + CollectionID: createCollectionMsg.Header().GetCollectionId(), + ChannelName: createCollectionMsg.VChannel(), + SeekPosition: &msgpb.MsgPosition{ + ChannelName: createCollectionMsg.VChannel(), + // from the last confirmed message id, you can read all messages which timetick is greater or equal than current message id. + MsgID: adaptor.MustGetMQWrapperIDFromMessage(createCollectionMsg.LastConfirmedMessageID()).Serialize(), + MsgGroup: "", // Not important any more. + Timestamp: createCollectionMsg.TimeTick(), + }, + }, + schema, + func(t syncmgr.Task, err error) { + if err != nil || t == nil { + return + } + if tt, ok := t.(*syncmgr.SyncTask); ok { + insertLogs, _, _, _ := tt.Binlogs() + resource.Resource().SegmentAssignStatsManager().UpdateOnSync(tt.SegmentID(), stats.SyncOperationMetrics{ + BinLogCounterIncr: 1, + BinLogFileCounterIncr: uint64(len(insertLogs)), + }) + } + }, + nil, + ) + impl.addNewDataSyncService(createCollectionMsg, msgChan, ds) + ds.Start() + impl.logger.Info("create data sync service done", zap.String("vchannel", createCollectionMsg.VChannel())) +} + +// WhenDropCollection handles the drop collection message. +func (impl *flusherComponents) WhenDropCollection(vchannel string) { + // flowgraph is removed by data sync service it self. + if ds, ok := impl.dataServices[vchannel]; ok { + ds.Close() + delete(impl.dataServices, vchannel) + impl.logger.Info("drop data sync service", zap.String("vchannel", vchannel)) + } +} + +func (impl *flusherComponents) HandleMessage(ctx context.Context, msg message.ImmutableMessage) error { + vchannel := msg.VChannel() + if vchannel == "" { + return impl.broadcastToAllDataSyncService(ctx, msg) + } + if _, ok := impl.dataServices[vchannel]; !ok { + return nil + } + return impl.dataServices[vchannel].HandleMessage(ctx, msg) +} + +func (impl *flusherComponents) broadcastToAllDataSyncService(ctx context.Context, msg message.ImmutableMessage) error { + for _, ds := range impl.dataServices { + if err := ds.HandleMessage(ctx, msg); err != nil { + return err + } + } + return nil +} + +func (impl *flusherComponents) addNewDataSyncService( + createCollectionMsg message.ImmutableCreateCollectionMessageV1, + input chan<- *msgstream.MsgPack, + ds *pipeline.DataSyncService, +) { + impl.dataServices[createCollectionMsg.VChannel()] = &dataService{ + input: input, + handler: adaptor.NewBaseMsgPackAdaptorHandler(), + ds: ds, + startMessageID: createCollectionMsg.LastConfirmedMessageID(), + } + impl.fgMgr.AddFlowgraph(ds) +} + +// Close release all the resources of components. +func (impl *flusherComponents) Close() { + impl.fgMgr.ClearFlowgraphs() + impl.wbMgr.Stop() + impl.cpUpdater.Close() + for _, ds := range impl.dataServices { + ds.Close() + } + impl.syncMgr.Close() +} + +// recover recover the components of the flusher. +func (impl *flusherComponents) recover(ctx context.Context, recoverInfos map[string]*datapb.GetChannelRecoveryInfoResponse) error { + futures := make(map[string]*conc.Future[interface{}], len(recoverInfos)) + for vchannel, recoverInfo := range recoverInfos { + recoverInfo := recoverInfo + future := GetExecPool().Submit(func() (interface{}, error) { + return impl.buildDataSyncServiceWithRetry(ctx, recoverInfo) + }) + futures[vchannel] = future + } + dataServices := make(map[string]*dataService, len(futures)) + var firstErr error + for vchannel, future := range futures { + ds, err := future.Await() + if err == nil { + dataServices[vchannel] = ds.(*dataService) + continue + } + if firstErr == nil { + firstErr = err + } + } + impl.dataServices = dataServices + return firstErr +} + +func (impl *flusherComponents) buildDataSyncServiceWithRetry(ctx context.Context, recoverInfo *datapb.GetChannelRecoveryInfoResponse) (*dataService, error) { + var ds *dataService + err := retry.Do(ctx, func() error { + var err error + ds, err = impl.buildDataSyncService(ctx, recoverInfo) + return err + }, retry.AttemptAlways()) + if err != nil { + return nil, err + } + return ds, nil +} + +func (impl *flusherComponents) buildDataSyncService(ctx context.Context, recoverInfo *datapb.GetChannelRecoveryInfoResponse) (*dataService, error) { + // Build and add pipeline. + input := make(chan *msgstream.MsgPack, 10) + ds, err := pipeline.NewStreamingNodeDataSyncService(ctx, + &util.PipelineParams{ + Ctx: context.Background(), + Broker: impl.broker, + SyncMgr: impl.syncMgr, + ChunkManager: impl.chunkManager, + WriteBufferManager: impl.wbMgr, + CheckpointUpdater: impl.cpUpdater, + Allocator: idalloc.NewMAllocator(resource.Resource().IDAllocator()), + MsgHandler: newMsgHandler(impl.wbMgr), + }, + &datapb.ChannelWatchInfo{Vchan: recoverInfo.GetInfo(), Schema: recoverInfo.GetSchema()}, + input, + func(t syncmgr.Task, err error) { + if err != nil || t == nil { + return + } + if tt, ok := t.(*syncmgr.SyncTask); ok { + insertLogs, _, _, _ := tt.Binlogs() + resource.Resource().SegmentAssignStatsManager().UpdateOnSync(tt.SegmentID(), stats.SyncOperationMetrics{ + BinLogCounterIncr: 1, + BinLogFileCounterIncr: uint64(len(insertLogs)), + }) + } + }, + nil, + ) + if err != nil { + return nil, err + } + return &dataService{ + input: input, + handler: adaptor.NewBaseMsgPackAdaptorHandler(), + ds: ds, + startMessageID: adaptor.MustGetMessageIDFromMQWrapperIDBytes(impl.wal.WALName(), recoverInfo.GetInfo().GetSeekPosition().GetMsgID()), + }, nil +} diff --git a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go deleted file mode 100644 index 677896997ce16..0000000000000 --- a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go +++ /dev/null @@ -1,231 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package flusherimpl - -import ( - "context" - "time" - - "github.com/cockroachdb/errors" - "go.uber.org/atomic" - "go.uber.org/zap" - - "github.com/milvus-io/milvus/internal/flushcommon/broker" - "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/storage" - "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/internal/util/idalloc" - "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" - "github.com/milvus-io/milvus/pkg/util/conc" - "github.com/milvus-io/milvus/pkg/util/funcutil" - "github.com/milvus-io/milvus/pkg/util/merr" - "github.com/milvus-io/milvus/pkg/util/paramtable" - "github.com/milvus-io/milvus/pkg/util/syncutil" - "github.com/milvus-io/milvus/pkg/util/typeutil" -) - -var _ flusher.Flusher = (*flusherImpl)(nil) - -type flusherImpl struct { - fgMgr pipeline.FlowgraphManager - wbMgr writebuffer.BufferManager - syncMgr syncmgr.SyncManager - cpUpdater *syncutil.Future[*util.ChannelCheckpointUpdater] - chunkManager storage.ChunkManager - - channelLifetimes *typeutil.ConcurrentMap[string, ChannelLifetime] - - notifyCh chan struct{} - notifier *syncutil.AsyncTaskNotifier[struct{}] -} - -func NewFlusher(chunkManager storage.ChunkManager) flusher.Flusher { - syncMgr := syncmgr.NewSyncManager(chunkManager) - wbMgr := writebuffer.NewManager(syncMgr) - return &flusherImpl{ - fgMgr: pipeline.NewFlowgraphManager(), - wbMgr: wbMgr, - syncMgr: syncMgr, - cpUpdater: syncutil.NewFuture[*util.ChannelCheckpointUpdater](), - chunkManager: chunkManager, - channelLifetimes: typeutil.NewConcurrentMap[string, ChannelLifetime](), - notifyCh: make(chan struct{}, 1), - notifier: syncutil.NewAsyncTaskNotifier[struct{}](), - } -} - -func (f *flusherImpl) RegisterPChannel(pchannel string, wal wal.WAL) error { - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - rc, err := resource.Resource().RootCoordClient().GetWithContext(ctx) - if err != nil { - return errors.Wrap(err, "At Get RootCoordClient") - } - resp, err := rc.GetPChannelInfo(ctx, &rootcoordpb.GetPChannelInfoRequest{ - Pchannel: pchannel, - }) - if err = merr.CheckRPCCall(resp, err); err != nil { - return err - } - for _, collectionInfo := range resp.GetCollections() { - f.RegisterVChannel(collectionInfo.GetVchannel(), wal) - } - return nil -} - -func (f *flusherImpl) RegisterVChannel(vchannel string, wal wal.WAL) { - _, ok := f.channelLifetimes.GetOrInsert(vchannel, NewChannelLifetime(f, vchannel, wal)) - if !ok { - log.Info("flusher register vchannel done", zap.String("vchannel", vchannel)) - } - f.notify() -} - -func (f *flusherImpl) UnregisterPChannel(pchannel string) { - f.channelLifetimes.Range(func(vchannel string, _ ChannelLifetime) bool { - if funcutil.ToPhysicalChannel(vchannel) == pchannel { - f.UnregisterVChannel(vchannel) - } - return true - }) -} - -func (f *flusherImpl) UnregisterVChannel(vchannel string) { - if clt, ok := f.channelLifetimes.GetAndRemove(vchannel); ok { - clt.Cancel() - } -} - -func (f *flusherImpl) notify() { - select { - case f.notifyCh <- struct{}{}: - default: - } -} - -func (f *flusherImpl) Start() { - f.wbMgr.Start() - go func() { - defer f.notifier.Finish(struct{}{}) - dc, err := resource.Resource().DataCoordClient().GetWithContext(f.notifier.Context()) - if err != nil { - return - } - broker := broker.NewCoordBroker(dc, paramtable.GetNodeID()) - cpUpdater := util.NewChannelCheckpointUpdater(broker) - go cpUpdater.Start() - f.cpUpdater.Set(cpUpdater) - - backoff := typeutil.NewBackoffTimer(typeutil.BackoffTimerConfig{ - Default: 5 * time.Second, - Backoff: typeutil.BackoffConfig{ - InitialInterval: 50 * time.Millisecond, - Multiplier: 2.0, - MaxInterval: 5 * time.Second, - }, - }) - - var nextTimer <-chan time.Time - for { - select { - case <-f.notifier.Context().Done(): - log.Info("flusher exited") - return - case <-f.notifyCh: - nextTimer = f.handle(backoff) - case <-nextTimer: - nextTimer = f.handle(backoff) - } - } - }() -} - -func (f *flusherImpl) handle(backoff *typeutil.BackoffTimer) <-chan time.Time { - futures := make([]*conc.Future[any], 0) - failureCnt := atomic.NewInt64(0) - f.channelLifetimes.Range(func(vchannel string, lifetime ChannelLifetime) bool { - future := GetExecPool().Submit(func() (any, error) { - err := lifetime.Run() - if errors.Is(err, errChannelLifetimeUnrecoverable) { - log.Warn("channel lifetime is unrecoverable, removed", zap.String("vchannel", vchannel)) - f.channelLifetimes.Remove(vchannel) - return nil, nil - } - if err != nil { - log.Warn("build pipeline failed", zap.String("vchannel", vchannel), zap.Error(err)) - failureCnt.Inc() - return nil, err - } - return nil, nil - }) - futures = append(futures, future) - return true - }) - _ = conc.BlockOnAll(futures...) - - if failureCnt.Load() > 0 { - backoff.EnableBackoff() - nextTimer, interval := backoff.NextTimer() - log.Warn("flusher lifetime trasition failed, retry with backoff...", zap.Int64("failureCnt", failureCnt.Load()), zap.Duration("interval", interval)) - return nextTimer - } - // There's a failure, do no backoff. - backoff.DisableBackoff() - return nil -} - -func (f *flusherImpl) Stop() { - f.notifier.Cancel() - f.notifier.BlockUntilFinish() - f.channelLifetimes.Range(func(vchannel string, lifetime ChannelLifetime) bool { - lifetime.Cancel() - return true - }) - f.fgMgr.ClearFlowgraphs() - f.wbMgr.Stop() - if f.cpUpdater.Ready() { - f.cpUpdater.Get().Close() - } -} - -func (f *flusherImpl) getPipelineParams(ctx context.Context) (*util.PipelineParams, error) { - dc, err := resource.Resource().DataCoordClient().GetWithContext(ctx) - if err != nil { - return nil, err - } - - cpUpdater, err := f.cpUpdater.GetWithContext(ctx) - if err != nil { - return nil, err - } - return &util.PipelineParams{ - Ctx: context.Background(), - Broker: broker.NewCoordBroker(dc, paramtable.GetNodeID()), - SyncMgr: f.syncMgr, - ChunkManager: f.chunkManager, - WriteBufferManager: f.wbMgr, - CheckpointUpdater: cpUpdater, - Allocator: idalloc.NewMAllocator(resource.Resource().IDAllocator()), - MsgHandler: newMsgHandler(f.wbMgr), - }, nil -} diff --git a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go deleted file mode 100644 index 05f3a2f704f32..0000000000000 --- a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go +++ /dev/null @@ -1,254 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package flusherimpl - -import ( - "context" - "sync" - "testing" - "time" - - "github.com/samber/lo" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" - "go.uber.org/atomic" - "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/mocks" - "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_wal" - "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/internal/types" - "github.com/milvus-io/milvus/pkg/common" - "github.com/milvus-io/milvus/pkg/proto/datapb" - "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" - "github.com/milvus-io/milvus/pkg/util/merr" - "github.com/milvus-io/milvus/pkg/util/paramtable" - "github.com/milvus-io/milvus/pkg/util/syncutil" -) - -func init() { - paramtable.Init() -} - -func newMockDatacoord(t *testing.T, maybe bool) *mocks.MockDataCoordClient { - datacoord := mocks.NewMockDataCoordClient(t) - failureCnt := atomic.NewInt32(20) - expect := datacoord.EXPECT().GetChannelRecoveryInfo(mock.Anything, mock.Anything).RunAndReturn( - func(ctx context.Context, request *datapb.GetChannelRecoveryInfoRequest, option ...grpc.CallOption, - ) (*datapb.GetChannelRecoveryInfoResponse, error) { - if failureCnt.Dec() > 0 { - return &datapb.GetChannelRecoveryInfoResponse{ - Status: merr.Status(merr.ErrCollectionNotFound), - }, nil - } - 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 - }) - if maybe { - expect.Maybe() - } - return datacoord -} - -func newMockWAL(t *testing.T, vchannels []string, maybe bool) *mock_wal.MockWAL { - w := mock_wal.NewMockWAL(t) - walName := w.EXPECT().WALName().Return("rocksmq") - if maybe { - walName.Maybe() - } - for range vchannels { - read := w.EXPECT().Read(mock.Anything, mock.Anything).RunAndReturn( - func(ctx context.Context, option wal.ReadOption) (wal.Scanner, error) { - handler := option.MesasgeHandler - scanner := mock_wal.NewMockScanner(t) - scanner.EXPECT().Close().RunAndReturn(func() error { - handler.Close() - return nil - }) - return scanner, nil - }) - if maybe { - read.Maybe() - } - } - return w -} - -func newTestFlusher(t *testing.T, maybe bool) flusher.Flusher { - m := mocks.NewChunkManager(t) - return NewFlusher(m) -} - -func TestFlusher_RegisterPChannel(t *testing.T) { - const ( - pchannel = "by-dev-rootcoord-dml_0" - maybe = false - ) - vchannels := []string{ - "by-dev-rootcoord-dml_0_123456v0", - "by-dev-rootcoord-dml_0_123456v1", - "by-dev-rootcoord-dml_0_123456v2", - } - - collectionsInfo := lo.Map(vchannels, func(vchannel string, i int) *rootcoordpb.CollectionInfoOnPChannel { - return &rootcoordpb.CollectionInfoOnPChannel{ - CollectionId: int64(i), - Partitions: []*rootcoordpb.PartitionInfoOnPChannel{{PartitionId: int64(i)}}, - Vchannel: vchannel, - } - }) - rootcoord := mocks.NewMockRootCoordClient(t) - rootcoord.EXPECT().GetPChannelInfo(mock.Anything, mock.Anything). - Return(&rootcoordpb.GetPChannelInfoResponse{Collections: collectionsInfo}, nil) - datacoord := newMockDatacoord(t, maybe) - - fDatacoord := syncutil.NewFuture[types.DataCoordClient]() - fDatacoord.Set(datacoord) - - fRootcoord := syncutil.NewFuture[types.RootCoordClient]() - fRootcoord.Set(rootcoord) - resource.InitForTest( - t, - resource.OptRootCoordClient(fRootcoord), - resource.OptDataCoordClient(fDatacoord), - ) - - f := newTestFlusher(t, maybe) - f.Start() - defer f.Stop() - - w := newMockWAL(t, vchannels, maybe) - err := f.RegisterPChannel(pchannel, w) - assert.NoError(t, err) - - assert.Eventually(t, func() bool { - return lo.EveryBy(vchannels, func(vchannel string) bool { - return f.(*flusherImpl).fgMgr.HasFlowgraph(vchannel) - }) - }, 10*time.Second, 10*time.Millisecond) - - f.UnregisterPChannel(pchannel) - assert.Equal(t, 0, f.(*flusherImpl).fgMgr.GetFlowgraphCount()) - assert.Equal(t, 0, f.(*flusherImpl).channelLifetimes.Len()) -} - -func TestFlusher_RegisterVChannel(t *testing.T) { - const ( - maybe = false - ) - vchannels := []string{ - "by-dev-rootcoord-dml_0_123456v0", - "by-dev-rootcoord-dml_0_123456v1", - "by-dev-rootcoord-dml_0_123456v2", - } - - datacoord := newMockDatacoord(t, maybe) - fDatacoord := syncutil.NewFuture[types.DataCoordClient]() - fDatacoord.Set(datacoord) - resource.InitForTest( - t, - resource.OptDataCoordClient(fDatacoord), - ) - - f := newTestFlusher(t, maybe) - f.Start() - defer f.Stop() - - w := newMockWAL(t, vchannels, maybe) - for _, vchannel := range vchannels { - f.RegisterVChannel(vchannel, w) - } - - assert.Eventually(t, func() bool { - return lo.EveryBy(vchannels, func(vchannel string) bool { - return f.(*flusherImpl).fgMgr.HasFlowgraph(vchannel) - }) - }, 10*time.Second, 10*time.Millisecond) - - for _, vchannel := range vchannels { - f.UnregisterVChannel(vchannel) - } - assert.Equal(t, 0, f.(*flusherImpl).fgMgr.GetFlowgraphCount()) - assert.Equal(t, 0, f.(*flusherImpl).channelLifetimes.Len()) -} - -func TestFlusher_Concurrency(t *testing.T) { - const ( - maybe = true - ) - vchannels := []string{ - "by-dev-rootcoord-dml_0_123456v0", - "by-dev-rootcoord-dml_0_123456v1", - "by-dev-rootcoord-dml_0_123456v2", - } - - datacoord := newMockDatacoord(t, maybe) - fDatacoord := syncutil.NewFuture[types.DataCoordClient]() - fDatacoord.Set(datacoord) - resource.InitForTest( - t, - resource.OptDataCoordClient(fDatacoord), - ) - - f := newTestFlusher(t, maybe) - f.Start() - defer f.Stop() - - w := newMockWAL(t, vchannels, maybe) - wg := &sync.WaitGroup{} - for i := 0; i < 10; i++ { - for _, vchannel := range vchannels { - wg.Add(1) - go func(vchannel string) { - f.RegisterVChannel(vchannel, w) - wg.Done() - }(vchannel) - } - for _, vchannel := range vchannels { - wg.Add(1) - go func(vchannel string) { - f.UnregisterVChannel(vchannel) - wg.Done() - }(vchannel) - } - } - wg.Wait() - - for _, vchannel := range vchannels { - f.UnregisterVChannel(vchannel) - } - - assert.Equal(t, 0, f.(*flusherImpl).fgMgr.GetFlowgraphCount()) - assert.Equal(t, 0, f.(*flusherImpl).channelLifetimes.Len()) -} diff --git a/internal/streamingnode/server/flusher/flusherimpl/pool.go b/internal/streamingnode/server/flusher/flusherimpl/pool.go index fcf527da2dcbe..9cf5a9b711381 100644 --- a/internal/streamingnode/server/flusher/flusherimpl/pool.go +++ b/internal/streamingnode/server/flusher/flusherimpl/pool.go @@ -23,18 +23,18 @@ import ( ) var ( - execPool *conc.Pool[any] + execPool *conc.Pool[interface{}] execPoolInitOnce sync.Once ) func initExecPool() { - execPool = conc.NewPool[any]( + execPool = conc.NewPool[interface{}]( 128, conc.WithPreAlloc(true), ) } -func GetExecPool() *conc.Pool[any] { +func GetExecPool() *conc.Pool[interface{}] { execPoolInitOnce.Do(initExecPool) return execPool } diff --git a/internal/streamingnode/server/flusher/flusherimpl/util.go b/internal/streamingnode/server/flusher/flusherimpl/util.go new file mode 100644 index 0000000000000..fb4bdf0c46959 --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/util.go @@ -0,0 +1,96 @@ +package flusherimpl + +import ( + "context" + "math" + + "github.com/cockroachdb/errors" + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/streamingnode/server/resource" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/util/conc" + "github.com/milvus-io/milvus/pkg/util/merr" + "github.com/milvus-io/milvus/pkg/util/retry" +) + +// getVchannels gets the vchannels of current pchannel. +func (impl *WALFlusherImpl) getVchannels(ctx context.Context, pchannel string) ([]string, error) { + var vchannels []string + rc, err := resource.Resource().RootCoordClient().GetWithContext(ctx) + if err != nil { + return nil, errors.Wrap(err, "when wait for rootcoord client ready") + } + retryCnt := -1 + if err := retry.Do(ctx, func() error { + retryCnt++ + resp, err := rc.GetPChannelInfo(ctx, &rootcoordpb.GetPChannelInfoRequest{ + Pchannel: pchannel, + }) + if err = merr.CheckRPCCall(resp, err); err != nil { + log.Warn("get pchannel info failed", zap.Error(err), zap.Int("retryCnt", retryCnt)) + return err + } + for _, collection := range resp.GetCollections() { + vchannels = append(vchannels, collection.Vchannel) + } + return nil + }, retry.AttemptAlways()); err != nil { + return nil, errors.Wrapf(err, "when get existed vchannels of pchannel") + } + return vchannels, nil +} + +// getRecoveryInfos gets the recovery info of the vchannels. +func (impl *WALFlusherImpl) getRecoveryInfos(ctx context.Context, vchannel []string) (map[string]*datapb.GetChannelRecoveryInfoResponse, error) { + futures := make([]*conc.Future[interface{}], 0, len(vchannel)) + for _, v := range vchannel { + v := v + future := GetExecPool().Submit(func() (interface{}, error) { + return impl.getRecoveryInfo(ctx, v) + }) + futures = append(futures, future) + } + recoveryInfos := make(map[string]*datapb.GetChannelRecoveryInfoResponse, len(futures)) + for i, future := range futures { + resp, err := future.Await() + if err == nil { + recoveryInfos[vchannel[i]] = resp.(*datapb.GetChannelRecoveryInfoResponse) + continue + } + if errors.Is(err, errChannelLifetimeUnrecoverable) { + impl.logger.Warn("channel has been dropped, skip to recover flusher for vchannel", zap.String("vchannel", vchannel[i])) + continue + } + return nil, errors.Wrapf(err, "when get recovery info of vchannel %s", vchannel[i]) + } + return recoveryInfos, nil +} + +// getRecoveryInfo gets the recovery info of the vchannel. +func (impl *WALFlusherImpl) getRecoveryInfo(ctx context.Context, vchannel string) (*datapb.GetChannelRecoveryInfoResponse, error) { + var resp *datapb.GetChannelRecoveryInfoResponse + retryCnt := -1 + err := retry.Do(ctx, func() error { + retryCnt++ + dc, err := resource.Resource().DataCoordClient().GetWithContext(ctx) + if err != nil { + // Should never failed at here. + return err + } + resp, err = dc.GetChannelRecoveryInfo(ctx, &datapb.GetChannelRecoveryInfoRequest{Vchannel: vchannel}) + if err = merr.CheckRPCCall(resp, err); err != nil { + impl.logger.Warn("get channel recovery info failed", zap.Error(err), zap.String("vchannel", vchannel), zap.Int("retryCnt", retryCnt)) + return err + } + // The channel has been dropped, skip to recover it. + if len(resp.GetInfo().GetSeekPosition().GetMsgID()) == 0 && resp.GetInfo().GetSeekPosition().GetTimestamp() == math.MaxUint64 { + impl.logger.Info("channel has been dropped, the vchannel can not be recovered", zap.String("vchannel", vchannel)) + return retry.Unrecoverable(errChannelLifetimeUnrecoverable) + } + return nil + }, retry.AttemptAlways()) + return resp, err +} diff --git a/internal/streamingnode/server/flusher/flusherimpl/wal_flusher.go b/internal/streamingnode/server/flusher/flusherimpl/wal_flusher.go new file mode 100644 index 0000000000000..34959e8ed9736 --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/wal_flusher.go @@ -0,0 +1,160 @@ +package flusherimpl + +import ( + "context" + + "github.com/cockroachdb/errors" + "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/streamingnode/server/wal/interceptors" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" + "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/retry" + "github.com/milvus-io/milvus/pkg/util/syncutil" +) + +var errChannelLifetimeUnrecoverable = errors.New("channel lifetime unrecoverable") + +// RecoverWALFlusher recovers the wal flusher. +func RecoverWALFlusher(param interceptors.InterceptorBuildParam) *WALFlusherImpl { + flusher := &WALFlusherImpl{ + notifier: syncutil.NewAsyncTaskNotifier[struct{}](), + wal: param.WAL, + logger: resource.Resource().Logger().With( + log.FieldComponent("flusher"), + zap.String("pchannel", param.WALImpls.Channel().Name)), + } + go flusher.Execute() + return flusher +} + +type WALFlusherImpl struct { + notifier *syncutil.AsyncTaskNotifier[struct{}] + wal *syncutil.Future[wal.WAL] + flusherComponents *flusherComponents + logger *log.MLogger +} + +// Execute starts the wal flusher. +func (impl *WALFlusherImpl) Execute() (err error) { + defer func() { + impl.notifier.Finish(struct{}{}) + if err == nil { + impl.logger.Info("wal flusher stop") + return + } + if !errors.Is(err, context.Canceled) { + impl.logger.DPanic("wal flusher stop to executing with unexpected error", zap.Error(err)) + return + } + impl.logger.Warn("wal flusher is canceled before executing", zap.Error(err)) + }() + + impl.logger.Info("wal flusher start to recovery...") + l, err := impl.wal.GetWithContext(impl.notifier.Context()) + if err != nil { + return errors.Wrap(err, "when get wal from future") + } + impl.logger.Info("wal ready for flusher recovery") + + impl.flusherComponents, err = impl.buildFlusherComponents(impl.notifier.Context(), l) + if err != nil { + return errors.Wrap(err, "when build flusher components") + } + defer impl.flusherComponents.Close() + + scanner, err := impl.generateScanner(impl.notifier.Context(), impl.wal.Get()) + if err != nil { + return errors.Wrap(err, "when generate scanner") + } + defer scanner.Close() + + impl.logger.Info("wal flusher start to work") + for { + select { + case <-impl.notifier.Context().Done(): + return nil + case msg, ok := <-scanner.Chan(): + if !ok { + impl.logger.Warn("wal flusher is closing for closed scanner channel, which is unexpected at graceful way") + return nil + } + if err := impl.dispatch(msg); err != nil { + // The error is always context canceled. + return nil + } + } + } +} + +func (impl *WALFlusherImpl) generateScanner(ctx context.Context, wal wal.WAL) (wal.Scanner, error) { + // If checkpoint is nil, + // there may be the first time the wal built or the older version milvus that doesn't have pchannel level checkpoint. + // Try to recover it from vchannels. + // TODO: retry infinitely. + var checkpoint *streamingpb.WALCheckpoint + retryCnt := -1 + if err := retry.Do(ctx, func() error { + var err error + retryCnt++ + if checkpoint, err = resource.Resource().StreamingNodeCatalog().GetConsumeCheckpoint(ctx, wal.Channel().Name); err != nil { + impl.logger.Warn("get consume checkpoint failed", zap.Error(err), zap.Int("retryCnt", retryCnt)) + return err + } + return nil + }, retry.AttemptAlways()); err != nil { + return nil, err + } + readOpt := impl.getReadOptions(checkpoint, wal.WALName()) + return wal.Read(ctx, readOpt) +} + +func (impl *WALFlusherImpl) getReadOptions(checkpoint *streamingpb.WALCheckpoint, walName string) wal.ReadOption { + handler := make(adaptor.ChanMessageHandler, 64) + readOpt := wal.ReadOption{ + VChannel: "", // We need consume all message from wal. + MesasgeHandler: handler, + DeliverPolicy: options.DeliverPolicyAll(), + } + if checkpoint != nil { + startMessageID := message.MustUnmarshalMessageID(walName, checkpoint.MessageID.Id) + impl.logger.Info("wal start to scan from pchannel checkpoint", zap.Stringer("startMessageID", startMessageID)) + readOpt.DeliverPolicy = options.DeliverPolicyStartFrom(startMessageID) + return readOpt + } + if startMessageID := impl.flusherComponents.GetMinimumStartMessage(); startMessageID != nil { + impl.logger.Info("wal start to scan from minimum vchannel checkpoint", zap.Stringer("startMessageID", startMessageID)) + readOpt.DeliverPolicy = options.DeliverPolicyStartAfter(startMessageID) + return readOpt + } + impl.logger.Info("wal start to scan from the beginning") + return readOpt +} + +// handleMessage handles the message from wal. +func (impl *WALFlusherImpl) dispatch(msg message.ImmutableMessage) error { + // Do the data sync service management here. + switch msg.MessageType() { + case message.MessageTypeCreateCollection: + createCollectionMsg, err := message.AsImmutableCreateCollectionMessageV1(msg) + if err != nil { + impl.logger.DPanic("the message type is not CreateCollectionMessage", zap.Error(err)) + return nil + } + impl.flusherComponents.WhenCreateCollection(createCollectionMsg) + case message.MessageTypeDropCollection: + // defer to remove the data sync service from the components. + defer impl.flusherComponents.WhenDropCollection(msg.VChannel()) + } + return impl.flusherComponents.HandleMessage(impl.notifier.Context(), msg) +} + +func (impl *WALFlusherImpl) Close() { + impl.notifier.Cancel() + impl.notifier.BlockUntilFinish() +} diff --git a/internal/streamingnode/server/resource/resource.go b/internal/streamingnode/server/resource/resource.go index cb762dccdb8b3..4d8d99be7e708 100644 --- a/internal/streamingnode/server/resource/resource.go +++ b/internal/streamingnode/server/resource/resource.go @@ -7,7 +7,6 @@ import ( "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/storage" - "github.com/milvus-io/milvus/internal/streamingnode/server/flusher" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment/stats" tinspector "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick/inspector" "github.com/milvus-io/milvus/internal/types" @@ -24,13 +23,6 @@ var r = &resourceImpl{ // 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 - } -} - // OptETCD provides the etcd client to the resource. func OptETCD(etcd *clientv3.Client) optResourceInit { return func(r *resourceImpl) { @@ -80,6 +72,7 @@ func Apply(opts ...optResourceInit) { func Done() { r.segmentAssignStatsManager = stats.NewStatsManager() r.timeTickInspector = tinspector.NewTimeTickSyncInspector() + assertNotNil(r.ChunkManager()) assertNotNil(r.TSOAllocator()) assertNotNil(r.RootCoordClient()) assertNotNil(r.DataCoordClient()) @@ -96,7 +89,6 @@ 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 logger *log.MLogger timestampAllocator idalloc.Allocator idAllocator idalloc.Allocator @@ -109,11 +101,6 @@ type resourceImpl struct { timeTickInspector tinspector.TimeTickSyncInspector } -// Flusher returns the flusher. -func (r *resourceImpl) Flusher() flusher.Flusher { - return r.flusher -} - // TSOAllocator returns the timestamp allocator to allocate timestamp. func (r *resourceImpl) TSOAllocator() idalloc.Allocator { return r.timestampAllocator diff --git a/internal/streamingnode/server/resource/resource_test.go b/internal/streamingnode/server/resource/resource_test.go index 8c219d86ff0c8..8d5feb6817858 100644 --- a/internal/streamingnode/server/resource/resource_test.go +++ b/internal/streamingnode/server/resource/resource_test.go @@ -7,6 +7,7 @@ import ( clientv3 "go.etcd.io/etcd/client/v3" "github.com/milvus-io/milvus/internal/mocks/mock_metastore" + "github.com/milvus-io/milvus/internal/mocks/mock_storage" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/syncutil" @@ -24,6 +25,7 @@ func TestApply(t *testing.T) { }) Apply( + OptChunkManager(mock_storage.NewMockChunkManager(t)), OptETCD(&clientv3.Client{}), OptRootCoordClient(syncutil.NewFuture[types.RootCoordClient]()), OptDataCoordClient(syncutil.NewFuture[types.DataCoordClient]()), diff --git a/internal/streamingnode/server/server.go b/internal/streamingnode/server/server.go index a5ce256e9ca43..0a59213d6778b 100644 --- a/internal/streamingnode/server/server.go +++ b/internal/streamingnode/server/server.go @@ -5,7 +5,7 @@ import ( "google.golang.org/grpc" - "github.com/milvus-io/milvus/internal/streamingnode/server/resource" + "github.com/milvus-io/milvus/internal/streamingnode/client/handler/registry" "github.com/milvus-io/milvus/internal/streamingnode/server/service" "github.com/milvus-io/milvus/internal/streamingnode/server/walmanager" "github.com/milvus-io/milvus/internal/util/sessionutil" @@ -44,8 +44,6 @@ func (s *Server) Init(ctx context.Context) (err error) { // Start starts the streamingnode server. func (s *Server) Start() { - resource.Resource().Flusher().Start() - log.Info("flusher started") } // Stop stops the streamingnode server. @@ -55,8 +53,6 @@ func (s *Server) Stop() { s.walManager.Close() log.Info("streamingnode server stopped") log.Info("stopping flusher...") - resource.Resource().Flusher().Stop() - log.Info("flusher stopped") } // initBasicComponent initialize all underlying dependency for streamingnode. @@ -66,6 +62,8 @@ func (s *Server) initBasicComponent(_ context.Context) { if err != nil { panic("open wal manager failed") } + // Register the wal manager to the local registry. + registry.RegisterLocalWALManager(s.walManager) } // initService initializes the grpc service. diff --git a/internal/streamingnode/server/wal/adaptor/builder.go b/internal/streamingnode/server/wal/adaptor/builder.go index 70e629ce7d6fe..659e86ad8e833 100644 --- a/internal/streamingnode/server/wal/adaptor/builder.go +++ b/internal/streamingnode/server/wal/adaptor/builder.go @@ -3,7 +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/flusher" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/redo" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick" @@ -34,8 +34,8 @@ func (b builderAdaptorImpl) Build() (wal.Opener, error) { // Add all interceptor here. return adaptImplsToOpener(o, []interceptors.InterceptorBuilder{ redo.NewInterceptorBuilder(), + flusher.NewInterceptorBuilder(), timetick.NewInterceptorBuilder(), segment.NewInterceptorBuilder(), - ddl.NewInterceptorBuilder(), }), nil } diff --git a/internal/streamingnode/server/wal/adaptor/message_handler.go b/internal/streamingnode/server/wal/adaptor/message_handler.go deleted file mode 100644 index 8ec28014a623b..0000000000000 --- a/internal/streamingnode/server/wal/adaptor/message_handler.go +++ /dev/null @@ -1,107 +0,0 @@ -package adaptor - -import ( - "github.com/milvus-io/milvus/internal/streamingnode/server/wal" - "github.com/milvus-io/milvus/pkg/mq/msgstream" - "github.com/milvus-io/milvus/pkg/streaming/util/message" - "github.com/milvus-io/milvus/pkg/streaming/util/message/adaptor" -) - -var ( - _ wal.MessageHandler = defaultMessageHandler(nil) - _ wal.MessageHandler = (*MsgPackAdaptorHandler)(nil) -) - -type defaultMessageHandler chan message.ImmutableMessage - -func (h defaultMessageHandler) Handle(param wal.HandleParam) wal.HandleResult { - var sendingCh chan message.ImmutableMessage - if param.Message != nil { - sendingCh = h - } - select { - case <-param.Ctx.Done(): - return wal.HandleResult{Error: param.Ctx.Err()} - case msg, ok := <-param.Upstream: - if !ok { - return wal.HandleResult{Error: wal.ErrUpstreamClosed} - } - return wal.HandleResult{Incoming: msg} - case sendingCh <- param.Message: - return wal.HandleResult{MessageHandled: true} - case <-param.TimeTickChan: - return wal.HandleResult{TimeTickUpdated: true} - } -} - -func (d defaultMessageHandler) Close() { - close(d) -} - -// NewMsgPackAdaptorHandler create a new message pack adaptor handler. -func NewMsgPackAdaptorHandler() *MsgPackAdaptorHandler { - return &MsgPackAdaptorHandler{ - base: adaptor.NewBaseMsgPackAdaptorHandler(), - } -} - -type MsgPackAdaptorHandler struct { - base *adaptor.BaseMsgPackAdaptorHandler -} - -// Chan is the channel for message. -func (m *MsgPackAdaptorHandler) Chan() <-chan *msgstream.MsgPack { - return m.base.Channel -} - -// Handle is the callback for handling message. -func (m *MsgPackAdaptorHandler) Handle(param wal.HandleParam) wal.HandleResult { - messageHandled := false - // not handle new message if there are pending msgPack. - if param.Message != nil && m.base.PendingMsgPack.Len() == 0 { - m.base.GenerateMsgPack(param.Message) - messageHandled = true - } - - for { - var sendCh chan<- *msgstream.MsgPack - if m.base.PendingMsgPack.Len() != 0 { - sendCh = m.base.Channel - } - - select { - case <-param.Ctx.Done(): - return wal.HandleResult{ - MessageHandled: messageHandled, - Error: param.Ctx.Err(), - } - case msg, notClose := <-param.Upstream: - if !notClose { - return wal.HandleResult{ - MessageHandled: messageHandled, - Error: wal.ErrUpstreamClosed, - } - } - return wal.HandleResult{ - Incoming: msg, - MessageHandled: messageHandled, - } - case sendCh <- m.base.PendingMsgPack.Next(): - m.base.PendingMsgPack.UnsafeAdvance() - if m.base.PendingMsgPack.Len() > 0 { - continue - } - return wal.HandleResult{MessageHandled: messageHandled} - case <-param.TimeTickChan: - return wal.HandleResult{ - MessageHandled: messageHandled, - TimeTickUpdated: true, - } - } - } -} - -// Close closes the handler. -func (m *MsgPackAdaptorHandler) Close() { - close(m.base.Channel) -} diff --git a/internal/streamingnode/server/wal/adaptor/message_handler_test.go b/internal/streamingnode/server/wal/adaptor/message_handler_test.go deleted file mode 100644 index b3c7dedafddda..0000000000000 --- a/internal/streamingnode/server/wal/adaptor/message_handler_test.go +++ /dev/null @@ -1,93 +0,0 @@ -package adaptor - -import ( - "context" - "testing" - - "github.com/stretchr/testify/assert" - - "github.com/milvus-io/milvus/internal/streamingnode/server/wal" - "github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message" - "github.com/milvus-io/milvus/pkg/streaming/util/message" - "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/rmq" -) - -func TestMsgPackAdaptorHandler(t *testing.T) { - messageID := rmq.NewRmqID(1) - tt := uint64(100) - msg := message.CreateTestInsertMessage( - t, - 1, - 1000, - tt, - messageID, - ) - immutableMsg := msg.IntoImmutableMessage(messageID) - - upstream := make(chan message.ImmutableMessage, 1) - - ctx := context.Background() - h := NewMsgPackAdaptorHandler() - done := make(chan struct{}) - go func() { - for range h.Chan() { - } - close(done) - }() - upstream <- immutableMsg - resp := h.Handle(wal.HandleParam{ - Ctx: ctx, - Upstream: upstream, - Message: nil, - }) - assert.Equal(t, resp.Incoming, immutableMsg) - assert.False(t, resp.MessageHandled) - assert.NoError(t, resp.Error) - - resp = h.Handle(wal.HandleParam{ - Ctx: ctx, - Upstream: upstream, - Message: resp.Incoming, - }) - assert.NoError(t, resp.Error) - assert.Nil(t, resp.Incoming) - assert.True(t, resp.MessageHandled) - h.Close() - - <-done -} - -func TestDefaultHandler(t *testing.T) { - h := make(defaultMessageHandler, 1) - done := make(chan struct{}) - go func() { - for range h { - } - close(done) - }() - - upstream := make(chan message.ImmutableMessage, 1) - msg := mock_message.NewMockImmutableMessage(t) - upstream <- msg - resp := h.Handle(wal.HandleParam{ - Ctx: context.Background(), - Upstream: upstream, - Message: nil, - }) - assert.NotNil(t, resp.Incoming) - assert.NoError(t, resp.Error) - assert.False(t, resp.MessageHandled) - assert.Equal(t, resp.Incoming, msg) - - resp = h.Handle(wal.HandleParam{ - Ctx: context.Background(), - Upstream: upstream, - Message: resp.Incoming, - }) - assert.NoError(t, resp.Error) - assert.Nil(t, resp.Incoming) - assert.True(t, resp.MessageHandled) - - h.Close() - <-done -} diff --git a/internal/streamingnode/server/wal/adaptor/scanner_adaptor.go b/internal/streamingnode/server/wal/adaptor/scanner_adaptor.go index 1e4844e6d7207..6befca3a078d7 100644 --- a/internal/streamingnode/server/wal/adaptor/scanner_adaptor.go +++ b/internal/streamingnode/server/wal/adaptor/scanner_adaptor.go @@ -1,21 +1,23 @@ package adaptor import ( + "context" + "sync" + + "github.com/cockroachdb/errors" "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/streamingnode/server/wal/interceptors/timetick" - "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick/inspector" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/metricsutil" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/utility" "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/streaming/util/types" "github.com/milvus-io/milvus/pkg/streaming/walimpls" "github.com/milvus-io/milvus/pkg/streaming/walimpls/helper" - "github.com/milvus-io/milvus/pkg/util/paramtable" ) var _ wal.Scanner = (*scannerAdaptorImpl)(nil) @@ -28,11 +30,8 @@ func newScannerAdaptor( scanMetrics *metricsutil.ScannerMetrics, cleanup func(), ) wal.Scanner { - if readOption.VChannel == "" { - panic("vchannel of scanner must be set") - } if readOption.MesasgeHandler == nil { - readOption.MesasgeHandler = defaultMessageHandler(make(chan message.ImmutableMessage)) + readOption.MesasgeHandler = adaptor.ChanMessageHandler(make(chan message.ImmutableMessage)) } options.GetFilterFunc(readOption.MessageFilter) logger := resource.Resource().Logger().With( @@ -41,35 +40,33 @@ func newScannerAdaptor( zap.String("channel", l.Channel().Name), ) s := &scannerAdaptorImpl{ - logger: logger, - innerWAL: l, - readOption: readOption, - filterFunc: options.GetFilterFunc(readOption.MessageFilter), - reorderBuffer: utility.NewReOrderBuffer(), - pendingQueue: utility.NewPendingQueue(), - txnBuffer: utility.NewTxnBuffer(logger, scanMetrics), - cleanup: cleanup, - ScannerHelper: helper.NewScannerHelper(name), - lastTimeTickInfo: inspector.TimeTickInfo{}, - metrics: scanMetrics, + logger: logger, + innerWAL: l, + readOption: readOption, + filterFunc: options.GetFilterFunc(readOption.MessageFilter), + reorderBuffer: utility.NewReOrderBuffer(), + pendingQueue: utility.NewPendingQueue(), + txnBuffer: utility.NewTxnBuffer(logger, scanMetrics), + cleanup: cleanup, + ScannerHelper: helper.NewScannerHelper(name), + metrics: scanMetrics, } - go s.executeConsume() + go s.execute() return s } // scannerAdaptorImpl is a wrapper of ScannerImpls to extend it into a Scanner interface. type scannerAdaptorImpl struct { *helper.ScannerHelper - logger *log.MLogger - innerWAL walimpls.WALImpls - readOption wal.ReadOption - filterFunc func(message.ImmutableMessage) bool - reorderBuffer *utility.ReOrderByTimeTickBuffer // only support time tick reorder now. - pendingQueue *utility.PendingQueue - txnBuffer *utility.TxnBuffer // txn buffer for txn message. - cleanup func() - lastTimeTickInfo inspector.TimeTickInfo - metrics *metricsutil.ScannerMetrics + logger *log.MLogger + innerWAL walimpls.WALImpls + readOption wal.ReadOption + filterFunc func(message.ImmutableMessage) bool + reorderBuffer *utility.ReOrderByTimeTickBuffer // support time tick reorder. + pendingQueue *utility.PendingQueue + txnBuffer *utility.TxnBuffer // txn buffer for txn message. + cleanup func() + metrics *metricsutil.ScannerMetrics } // Channel returns the channel assignment info of the wal. @@ -79,7 +76,7 @@ func (s *scannerAdaptorImpl) Channel() types.PChannelInfo { // Chan returns the message channel of the scanner. func (s *scannerAdaptorImpl) Chan() <-chan message.ImmutableMessage { - return s.readOption.MesasgeHandler.(defaultMessageHandler) + return s.readOption.MesasgeHandler.(adaptor.ChanMessageHandler) } // Close the scanner, release the underlying resources. @@ -93,33 +90,71 @@ func (s *scannerAdaptorImpl) Close() error { return err } -func (s *scannerAdaptorImpl) executeConsume() { - defer s.readOption.MesasgeHandler.Close() +func (s *scannerAdaptorImpl) execute() { + defer func() { + s.readOption.MesasgeHandler.Close() + s.Finish(nil) + s.logger.Info("scanner is closed") + }() + s.logger.Info("scanner start background task") + + msgChan := make(chan message.ImmutableMessage) + wg := sync.WaitGroup{} + wg.Add(2) + go func() { + defer wg.Done() + err := s.produceEventLoop(msgChan) + if errors.Is(err, context.Canceled) { + s.logger.Info("the produce event loop of scanner is closed") + return + } + s.logger.Warn("the produce event loop of scanner is closed with unexpected error", zap.Error(err)) + }() + go func() { + defer wg.Done() + err := s.consumeEventLoop(msgChan) + if errors.Is(err, context.Canceled) { + s.logger.Info("the consuming event loop of scanner is closed") + return + } + s.logger.Warn("the consuming event loop of scanner is closed with unexpected error", zap.Error(err)) + }() + wg.Wait() +} - innerScanner, err := s.innerWAL.Read(s.Context(), walimpls.ReadOption{ - Name: s.Name(), - DeliverPolicy: s.readOption.DeliverPolicy, - }) +func (s *scannerAdaptorImpl) produceEventLoop(msgChan chan<- message.ImmutableMessage) error { + wb, err := resource.Resource().TimeTickInspector().MustGetOperator(s.Channel()).WriteAheadBuffer(s.Context()) if err != nil { - s.Finish(err) - return + return err } - defer innerScanner.Close() - timeTickNotifier := resource.Resource().TimeTickInspector().MustGetOperator(s.Channel()).TimeTickNotifier() + scanner := newSwithableScanner(s.Name(), s.logger, s.innerWAL, wb, s.readOption.DeliverPolicy, msgChan) + s.logger.Info("start produce loop of scanner at mode", zap.String("mode", scanner.Mode())) + for { + if scanner, err = scanner.Do(s.Context()); err != nil { + return err + } + s.logger.Info("switch scanner mode", zap.String("mode", scanner.Mode())) + } +} +func (s *scannerAdaptorImpl) consumeEventLoop(msgChan <-chan message.ImmutableMessage) error { for { + var upstream <-chan message.ImmutableMessage + if s.pendingQueue.Len() > 16 { + // If the pending queue is full, we need to wait until it's consumed to avoid scanner overloading. + upstream = nil + } else { + upstream = msgChan + } // generate the event channel and do the event loop. - // TODO: Consume from local cache. - handleResult := s.readOption.MesasgeHandler.Handle(wal.HandleParam{ - Ctx: s.Context(), - Upstream: s.getUpstream(innerScanner), - TimeTickChan: s.getTimeTickUpdateChan(timeTickNotifier), - Message: s.pendingQueue.Next(), + handleResult := s.readOption.MesasgeHandler.Handle(message.HandleParam{ + Ctx: s.Context(), + Upstream: upstream, + Message: s.pendingQueue.Next(), }) if handleResult.Error != nil { - s.Finish(handleResult.Error) - return + return handleResult.Error } if handleResult.MessageHandled { s.pendingQueue.UnsafeAdvance() @@ -128,30 +163,9 @@ func (s *scannerAdaptorImpl) executeConsume() { if handleResult.Incoming != nil { s.handleUpstream(handleResult.Incoming) } - // If the timetick just updated with a non persist operation, - // we just make a fake message to keep timetick sync if there are no more pending message. - if handleResult.TimeTickUpdated { - s.handleTimeTickUpdated(timeTickNotifier) - } } } -func (s *scannerAdaptorImpl) getTimeTickUpdateChan(timeTickNotifier *inspector.TimeTickNotifier) <-chan struct{} { - if s.pendingQueue.Len() == 0 && s.reorderBuffer.Len() == 0 && !s.lastTimeTickInfo.IsZero() { - return timeTickNotifier.WatchAtMessageID(s.lastTimeTickInfo.MessageID, s.lastTimeTickInfo.TimeTick) - } - return nil -} - -func (s *scannerAdaptorImpl) getUpstream(scanner walimpls.ScannerImpls) <-chan message.ImmutableMessage { - // TODO: configurable pending buffer count. - // If the pending queue is full, we need to wait until it's consumed to avoid scanner overloading. - if s.pendingQueue.Len() > 16 { - return nil - } - return scanner.Chan() -} - func (s *scannerAdaptorImpl) handleUpstream(msg message.ImmutableMessage) { // Observe the message. s.metrics.ObserveMessage(msg.MessageType(), msg.EstimateSize()) @@ -165,22 +179,24 @@ func (s *scannerAdaptorImpl) handleUpstream(msg message.ImmutableMessage) { msgs := s.txnBuffer.HandleImmutableMessages(messages, msg.TimeTick()) s.metrics.UpdateTxnBufSize(s.txnBuffer.Bytes()) - // Push the confirmed messages into pending queue for consuming. - // and push forward timetick info. - s.pendingQueue.Add(msgs) - s.metrics.UpdatePendingQueueSize(s.pendingQueue.Bytes()) - s.lastTimeTickInfo = inspector.TimeTickInfo{ - MessageID: msg.MessageID(), - TimeTick: msg.TimeTick(), - LastConfirmedMessageID: msg.LastConfirmedMessageID(), + if len(msgs) > 0 { + // Push the confirmed messages into pending queue for consuming. + s.pendingQueue.Add(msgs) + } else if s.pendingQueue.Len() == 0 { + // If there's no new message incoming and there's no pending message in the queue. + // Add current timetick message into pending queue to make timetick push forward. + // TODO: current milvus can only run on timetick pushing, + // after qview is applied, those trival time tick message can be erased. + s.pendingQueue.Add([]message.ImmutableMessage{msg}) } + s.metrics.UpdatePendingQueueSize(s.pendingQueue.Bytes()) return } // Filtering the vchannel // If the message is not belong to any vchannel, it should be broadcasted to all vchannels. // Otherwise, it should be filtered by vchannel. - if msg.VChannel() != "" && s.readOption.VChannel != msg.VChannel() { + if msg.VChannel() != "" && s.readOption.VChannel != "" && s.readOption.VChannel != msg.VChannel() { return } // Filtering the message if needed. @@ -190,7 +206,9 @@ func (s *scannerAdaptorImpl) handleUpstream(msg message.ImmutableMessage) { } // otherwise add message into reorder buffer directly. if err := s.reorderBuffer.Push(msg); err != nil { - s.metrics.ObserveTimeTickViolation(msg.MessageType()) + if errors.Is(err, utility.ErrTimeTickVoilation) { + s.metrics.ObserveTimeTickViolation(msg.MessageType()) + } s.logger.Warn("failed to push message into reorder buffer", zap.Any("msgID", msg.MessageID()), zap.Uint64("timetick", msg.TimeTick()), @@ -201,21 +219,3 @@ func (s *scannerAdaptorImpl) handleUpstream(msg message.ImmutableMessage) { s.metrics.UpdateTimeTickBufSize(s.reorderBuffer.Bytes()) s.metrics.ObserveFilteredMessage(msg.MessageType(), msg.EstimateSize()) } - -func (s *scannerAdaptorImpl) handleTimeTickUpdated(timeTickNotifier *inspector.TimeTickNotifier) { - timeTickInfo := timeTickNotifier.Get() - if timeTickInfo.MessageID.EQ(s.lastTimeTickInfo.MessageID) && timeTickInfo.TimeTick > s.lastTimeTickInfo.TimeTick { - s.lastTimeTickInfo.TimeTick = timeTickInfo.TimeTick - msg, err := timetick.NewTimeTickMsg( - s.lastTimeTickInfo.TimeTick, - s.lastTimeTickInfo.LastConfirmedMessageID, - paramtable.GetNodeID(), - ) - if err != nil { - s.logger.Warn("unreachable: a marshal timetick operation must be success") - return - } - s.pendingQueue.AddOne(msg.IntoImmutableMessage(s.lastTimeTickInfo.MessageID)) - s.metrics.UpdatePendingQueueSize(s.pendingQueue.Bytes()) - } -} diff --git a/internal/streamingnode/server/wal/adaptor/scanner_adaptor_test.go b/internal/streamingnode/server/wal/adaptor/scanner_adaptor_test.go index e80bfffddd1ad..aba0f4155b416 100644 --- a/internal/streamingnode/server/wal/adaptor/scanner_adaptor_test.go +++ b/internal/streamingnode/server/wal/adaptor/scanner_adaptor_test.go @@ -2,11 +2,15 @@ package adaptor import ( "testing" + "time" "github.com/cockroachdb/errors" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" + "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/wal/interceptors/mock_wab" + "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/wal/interceptors/timetick/mock_inspector" + "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/metricsutil" "github.com/milvus-io/milvus/pkg/mocks/streaming/mock_walimpls" @@ -15,21 +19,20 @@ import ( ) func TestScannerAdaptorReadError(t *testing.T) { + resource.InitForTest(t) + + operator := mock_inspector.NewMockTimeTickSyncOperator(t) + operator.EXPECT().Channel().Return(types.PChannelInfo{}) + operator.EXPECT().Sync(mock.Anything).Return() + wb := mock_wab.NewMockROWriteAheadBuffer(t) + operator.EXPECT().WriteAheadBuffer(mock.Anything).Return(wb, nil) + resource.Resource().TimeTickInspector().RegisterSyncOperator(operator) + err := errors.New("read error") l := mock_walimpls.NewMockWALImpls(t) l.EXPECT().Read(mock.Anything, mock.Anything).Return(nil, err) l.EXPECT().Channel().Return(types.PChannelInfo{}) - assert.Panics(t, func() { - s := newScannerAdaptor("scanner", l, - wal.ReadOption{ - DeliverPolicy: options.DeliverPolicyAll(), - MessageFilter: nil, - }, - metricsutil.NewScanMetrics(types.PChannelInfo{}).NewScannerMetrics(), - func() {}) - defer s.Close() - }) s := newScannerAdaptor("scanner", l, wal.ReadOption{ VChannel: "test", @@ -38,8 +41,9 @@ func TestScannerAdaptorReadError(t *testing.T) { }, metricsutil.NewScanMetrics(types.PChannelInfo{}).NewScannerMetrics(), func() {}) - defer s.Close() + time.Sleep(200 * time.Millisecond) + s.Close() <-s.Chan() <-s.Done() - assert.ErrorIs(t, s.Error(), err) + assert.NoError(t, s.Error()) } diff --git a/internal/streamingnode/server/wal/adaptor/scanner_switchable.go b/internal/streamingnode/server/wal/adaptor/scanner_switchable.go new file mode 100644 index 0000000000000..20ab374019936 --- /dev/null +++ b/internal/streamingnode/server/wal/adaptor/scanner_switchable.go @@ -0,0 +1,213 @@ +package adaptor + +import ( + "context" + "time" + + "github.com/cockroachdb/errors" + "go.uber.org/zap" + + "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/wab" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/options" + "github.com/milvus-io/milvus/pkg/streaming/walimpls" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +var ( + _ switchableScanner = (*tailingScanner)(nil) + _ switchableScanner = (*catchupScanner)(nil) +) + +// newSwitchableScanner creates a new switchable scanner. +func newSwithableScanner( + scannerName string, + logger *log.MLogger, + innerWAL walimpls.WALImpls, + writeAheadBuffer wab.ROWriteAheadBuffer, + deliverPolicy options.DeliverPolicy, + msgChan chan<- message.ImmutableMessage, +) switchableScanner { + return &catchupScanner{ + switchableScannerImpl: switchableScannerImpl{ + scannerName: scannerName, + logger: logger, + innerWAL: innerWAL, + msgChan: msgChan, + writeAheadBuffer: writeAheadBuffer, + }, + deliverPolicy: deliverPolicy, + exclusiveStartTimeTick: 0, + } +} + +// switchableScanner is a scanner that can switch between Catchup and Tailing mode +type switchableScanner interface { + // Mode is Catchup or Tailing + Mode() string + + // Execute make a scanner work at background. + // When the scanner want to change the mode, it will return a new scanner with the new mode. + // When error is returned, the scanner is canceled and unrecoverable forever. + Do(ctx context.Context) (switchableScanner, error) +} + +type switchableScannerImpl struct { + scannerName string + logger *log.MLogger + innerWAL walimpls.WALImpls + msgChan chan<- message.ImmutableMessage + writeAheadBuffer wab.ROWriteAheadBuffer +} + +func (s *switchableScannerImpl) HandleMessage(ctx context.Context, msg message.ImmutableMessage) error { + select { + case <-ctx.Done(): + return ctx.Err() + case s.msgChan <- msg: + return nil + } +} + +// catchupScanner is a scanner that make a read at underlying wal, and try to catchup the writeahead buffer then switch to tailing mode. +type catchupScanner struct { + switchableScannerImpl + deliverPolicy options.DeliverPolicy + exclusiveStartTimeTick uint64 // scanner should filter out the message that less than or equal to this time tick. +} + +func (s *catchupScanner) Mode() string { + return "Catchup" +} + +func (s *catchupScanner) Do(ctx context.Context) (switchableScanner, error) { + for { + if ctx.Err() != nil { + return nil, ctx.Err() + } + scanner, err := s.createReaderWithBackoff(ctx, s.deliverPolicy) + if err != nil { + // Only the cancellation error will be returned, other error will keep backoff. + return nil, err + } + switchedScanner, err := s.consumeWithScanner(ctx, scanner) + if err != nil { + s.logger.Warn("scanner consuming was interrpurted with error, start a backoff", zap.Error(err)) + continue + } + return switchedScanner, nil + } +} + +func (s *catchupScanner) consumeWithScanner(ctx context.Context, scanner walimpls.ScannerImpls) (switchableScanner, error) { + defer scanner.Close() + for { + select { + case <-ctx.Done(): + return nil, ctx.Err() + case msg, ok := <-scanner.Chan(): + if !ok { + return nil, scanner.Error() + } + if msg.TimeTick() <= s.exclusiveStartTimeTick { + // we should filter out the message that less than or equal to this time tick to remove duplicate message + // when we switch from tailing mode to catchup mode. + continue + } + if err := s.HandleMessage(ctx, msg); err != nil { + return nil, err + } + if msg.MessageType() != message.MessageTypeTimeTick { + continue + } + // Here's a timetick message from the scanner, Check if we catch up the writeahead buffer, make tailing read after that. + if reader, err := s.writeAheadBuffer.ReadFromExclusiveTimeTick(ctx, msg.TimeTick()); err == nil { + s.logger.Info( + "scanner consuming was interrpted because catup done", + zap.Uint64("timetick", msg.TimeTick()), + zap.Stringer("messageID", msg.MessageID()), + zap.Stringer("lastConfirmedMessageID", msg.LastConfirmedMessageID()), + ) + return &tailingScanner{ + switchableScannerImpl: s.switchableScannerImpl, + reader: reader, + lastConsumedMessage: msg, + }, nil + } + } + } +} + +func (s *catchupScanner) createReaderWithBackoff(ctx context.Context, deliverPolicy options.DeliverPolicy) (walimpls.ScannerImpls, error) { + backoffTimer := typeutil.NewBackoffTimer(typeutil.BackoffTimerConfig{ + Default: 5 * time.Second, + Backoff: typeutil.BackoffConfig{ + InitialInterval: 100 * time.Millisecond, + Multiplier: 2.0, + MaxInterval: 5 * time.Second, + }, + }) + backoffTimer.EnableBackoff() + for { + innerScanner, err := s.innerWAL.Read(ctx, walimpls.ReadOption{ + Name: s.scannerName, + DeliverPolicy: deliverPolicy, + }) + if err == nil { + return innerScanner, nil + } + if ctx.Err() != nil { + // The scanner is closing, so stop the backoff. + return nil, ctx.Err() + } + waker, nextInterval := backoffTimer.NextTimer() + s.logger.Warn("create underlying scanner for wal scanner, start a backoff", + zap.Duration("nextInterval", nextInterval), + zap.Error(err), + ) + select { + case <-ctx.Done(): + return nil, ctx.Err() + case <-waker: + } + } +} + +// tailingScanner is used to perform a tailing read from the writeaheadbuffer of wal. +type tailingScanner struct { + switchableScannerImpl + reader *wab.WriteAheadBufferReader + lastConsumedMessage message.ImmutableMessage +} + +func (s *tailingScanner) Mode() string { + return "Tailing" +} + +func (s *tailingScanner) Do(ctx context.Context) (switchableScanner, error) { + for { + msg, err := s.reader.Next(ctx) + if errors.Is(err, wab.ErrEvicted) { + // The tailing read is failure, switch into catchup mode. + s.logger.Info( + "scanner consuming was interrpted because tailing eviction", + zap.Uint64("timetick", s.lastConsumedMessage.TimeTick()), + zap.Stringer("messageID", s.lastConsumedMessage.MessageID()), + zap.Stringer("lastConfirmedMessageID", s.lastConsumedMessage.LastConfirmedMessageID()), + ) + return &catchupScanner{ + switchableScannerImpl: s.switchableScannerImpl, + deliverPolicy: options.DeliverPolicyStartFrom(s.lastConsumedMessage.LastConfirmedMessageID()), + exclusiveStartTimeTick: s.lastConsumedMessage.TimeTick(), + }, nil + } + if err != nil { + return nil, err + } + if err := s.HandleMessage(ctx, msg); err != nil { + return nil, err + } + s.lastConsumedMessage = msg + } +} diff --git a/internal/streamingnode/server/wal/adaptor/wal_adaptor.go b/internal/streamingnode/server/wal/adaptor/wal_adaptor.go index 17e0909b4e7a4..a30a94ba9372b 100644 --- a/internal/streamingnode/server/wal/adaptor/wal_adaptor.go +++ b/internal/streamingnode/server/wal/adaptor/wal_adaptor.go @@ -111,7 +111,6 @@ func (w *walAdaptorImpl) Append(ctx context.Context, msg message.MutableMessage) func(ctx context.Context, msg message.MutableMessage) (message.MessageID, error) { if notPersistHint := utility.GetNotPersisted(ctx); notPersistHint != nil { // do not persist the message if the hint is set. - // only used by time tick sync operator. return notPersistHint.MessageID, nil } metricsGuard.StartWALImplAppend() @@ -223,7 +222,7 @@ func (w *walAdaptorImpl) Close() { logger.Info("scanner close done, close inner wal...") w.inner.Close() - logger.Info("scanner close done, close interceptors...") + logger.Info("wal close done, close interceptors...") w.interceptorBuildResult.Close() w.appendExecutionPool.Free() diff --git a/internal/streamingnode/server/wal/adaptor/wal_adaptor_test.go b/internal/streamingnode/server/wal/adaptor/wal_adaptor_test.go index cdc82156881d8..96cb1a18d5241 100644 --- a/internal/streamingnode/server/wal/adaptor/wal_adaptor_test.go +++ b/internal/streamingnode/server/wal/adaptor/wal_adaptor_test.go @@ -10,12 +10,12 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" + "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/wal/interceptors/mock_wab" "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/wal/interceptors/timetick/mock_inspector" "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/wal/mock_interceptors" "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/internal/streamingnode/server/wal/interceptors/timetick/inspector" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/pkg/mocks/streaming/mock_walimpls" "github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message" @@ -26,6 +26,8 @@ import ( ) func TestWalAdaptorReadFail(t *testing.T) { + resource.InitForTest(t) + l := mock_walimpls.NewMockWALImpls(t) expectedErr := errors.New("test") l.EXPECT().WALName().Return("test") @@ -35,6 +37,15 @@ func TestWalAdaptorReadFail(t *testing.T) { return nil, expectedErr }) + writeAheadBuffer := mock_wab.NewMockROWriteAheadBuffer(t) + operator := mock_inspector.NewMockTimeTickSyncOperator(t) + operator.EXPECT().Channel().Return(types.PChannelInfo{}) + operator.EXPECT().Sync(mock.Anything).Return() + operator.EXPECT().WriteAheadBuffer(mock.Anything).Return(writeAheadBuffer, nil) + resource.Resource().TimeTickInspector().RegisterSyncOperator( + operator, + ) + lAdapted := adaptImplsToWAL(l, nil, func() {}) scanner, err := lAdapted.Read(context.Background(), wal.ReadOption{ VChannel: "test", @@ -48,7 +59,6 @@ func TestWALAdaptor(t *testing.T) { resource.InitForTest(t) operator := mock_inspector.NewMockTimeTickSyncOperator(t) - operator.EXPECT().TimeTickNotifier().Return(inspector.NewTimeTickNotifier()) operator.EXPECT().Channel().Return(types.PChannelInfo{}) operator.EXPECT().Sync(mock.Anything).Return() resource.Resource().TimeTickInspector().RegisterSyncOperator(operator) diff --git a/internal/streamingnode/server/wal/adaptor/wal_test.go b/internal/streamingnode/server/wal/adaptor/wal_test.go index 3b49bf1933882..c241a2acd559a 100644 --- a/internal/streamingnode/server/wal/adaptor/wal_test.go +++ b/internal/streamingnode/server/wal/adaptor/wal_test.go @@ -17,8 +17,6 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/mocks/mock_metastore" - "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_flusher" - "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/registry" internaltypes "github.com/milvus-io/milvus/internal/types" @@ -67,20 +65,6 @@ func initResourceForTest(t *testing.T) { catalog := mock_metastore.NewMockStreamingNodeCataLog(t) catalog.EXPECT().ListSegmentAssignment(mock.Anything, mock.Anything).Return(nil, nil) catalog.EXPECT().SaveSegmentAssignments(mock.Anything, mock.Anything, mock.Anything).Return(nil) - - flusher := mock_flusher.NewMockFlusher(t) - flusher.EXPECT().RegisterPChannel(mock.Anything, mock.Anything).Return(nil).Maybe() - flusher.EXPECT().UnregisterPChannel(mock.Anything).Return().Maybe() - flusher.EXPECT().RegisterVChannel(mock.Anything, mock.Anything).Return().Maybe() - flusher.EXPECT().UnregisterVChannel(mock.Anything).Return().Maybe() - - resource.InitForTest( - t, - resource.OptRootCoordClient(fRootCoordClient), - resource.OptDataCoordClient(fDataCoordClient), - resource.OptFlusher(flusher), - resource.OptStreamingNodeCatalog(catalog), - ) } func (f *walTestFramework) Run() { diff --git a/internal/streamingnode/server/wal/interceptors/ddl/builder.go b/internal/streamingnode/server/wal/interceptors/ddl/builder.go deleted file mode 100644 index d07ed3aed3abc..0000000000000 --- a/internal/streamingnode/server/wal/interceptors/ddl/builder.go +++ /dev/null @@ -1,39 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package 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.Interceptor { - interceptor := &ddlAppendInterceptor{ - wal: param.WAL, - } - return interceptor -} diff --git a/internal/streamingnode/server/wal/interceptors/ddl/ddl_interceptor.go b/internal/streamingnode/server/wal/interceptors/ddl/ddl_interceptor.go deleted file mode 100644 index 7cb1ee12384ab..0000000000000 --- a/internal/streamingnode/server/wal/interceptors/ddl/ddl_interceptor.go +++ /dev/null @@ -1,55 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package 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.Interceptor = (*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) (message.MessageID, error) { - // send the create collection message. - msgID, err := append(ctx, msg) - if err != nil { - return msgID, err - } - - switch msg.MessageType() { - case message.MessageTypeCreateCollection: - resource.Resource().Flusher().RegisterVChannel(msg.VChannel(), d.wal.Get()) - case message.MessageTypeDropCollection: - // TODO: unregister vchannel, cannot unregister vchannel now. - // Wait for PR: https://github.com/milvus-io/milvus/pull/37176 - } - return msgID, nil -} - -// Close implements BasicInterceptor. -func (d *ddlAppendInterceptor) Close() {} diff --git a/internal/streamingnode/server/wal/interceptors/flusher/builder.go b/internal/streamingnode/server/wal/interceptors/flusher/builder.go new file mode 100644 index 0000000000000..1d54221c0d3a7 --- /dev/null +++ b/internal/streamingnode/server/wal/interceptors/flusher/builder.go @@ -0,0 +1,22 @@ +package flusher + +import ( + "github.com/milvus-io/milvus/internal/streamingnode/server/flusher/flusherimpl" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors" +) + +// NewInterceptorBuilder creates a new flusher interceptor builder. +func NewInterceptorBuilder() interceptors.InterceptorBuilder { + return &interceptorBuilder{} +} + +// interceptorBuilder is the builder for flusher interceptor. +type interceptorBuilder struct{} + +// Build creates a new flusher interceptor. +func (b *interceptorBuilder) Build(param interceptors.InterceptorBuildParam) interceptors.Interceptor { + flusher := flusherimpl.RecoverWALFlusher(param) + return &flusherAppendInterceptor{ + flusher: flusher, + } +} diff --git a/internal/streamingnode/server/wal/interceptors/flusher/flusher_interceptor.go b/internal/streamingnode/server/wal/interceptors/flusher/flusher_interceptor.go new file mode 100644 index 0000000000000..0749c04af466b --- /dev/null +++ b/internal/streamingnode/server/wal/interceptors/flusher/flusher_interceptor.go @@ -0,0 +1,33 @@ +package flusher + +import ( + "context" + + "github.com/milvus-io/milvus/internal/streamingnode/server/flusher/flusherimpl" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors" + "github.com/milvus-io/milvus/pkg/streaming/util/message" +) + +var ( + _ interceptors.Interceptor = (*flusherAppendInterceptor)(nil) + _ interceptors.InterceptorWithGracefulClose = (*flusherAppendInterceptor)(nil) +) + +// flusherAppendInterceptor is an append interceptor to handle the append operation from consumer. +// the flusher is a unique consumer that will consume the message from wal. +// It will handle the message and persist the message other storage from wal. +type flusherAppendInterceptor struct { + flusher *flusherimpl.WALFlusherImpl +} + +func (c *flusherAppendInterceptor) DoAppend(ctx context.Context, msg message.MutableMessage, append interceptors.Append) (msgID message.MessageID, err error) { + // TODO: The interceptor will also do some slow down for streaming service if the consumer is lag too much. + return append(ctx, msg) +} + +// GracefulClose will close the flusher gracefully. +func (c *flusherAppendInterceptor) GracefulClose() { + c.flusher.Close() +} + +func (c *flusherAppendInterceptor) Close() {} diff --git a/internal/streamingnode/server/wal/interceptors/segment/inspector/impls.go b/internal/streamingnode/server/wal/interceptors/segment/inspector/impls.go index 6bc8569b15969..22de69e39a6fd 100644 --- a/internal/streamingnode/server/wal/interceptors/segment/inspector/impls.go +++ b/internal/streamingnode/server/wal/interceptors/segment/inspector/impls.go @@ -129,7 +129,7 @@ func (s *sealOperationInspectorImpl) background() { if segmentBelongs == nil { continue } - log.Info("seal by total growing segments size", zap.String("vchannel", segmentBelongs.VChannel), + s.logger.Info("seal by total growing segments size", zap.String("vchannel", segmentBelongs.VChannel), zap.Uint64("sealThreshold", threshold), zap.Int64("sealSegment", segmentBelongs.SegmentID)) if pm, ok := s.managers.Get(segmentBelongs.PChannel); ok { diff --git a/internal/streamingnode/server/wal/interceptors/timetick/ack/ack_test.go b/internal/streamingnode/server/wal/interceptors/timetick/ack/ack_test.go index 86ce5eed5385c..d6d641f442d99 100644 --- a/internal/streamingnode/server/wal/interceptors/timetick/ack/ack_test.go +++ b/internal/streamingnode/server/wal/interceptors/timetick/ack/ack_test.go @@ -17,6 +17,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/metricsutil" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message" "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" "github.com/milvus-io/milvus/pkg/util/merr" @@ -200,8 +201,10 @@ func TestAckManager(t *testing.T) { time.Sleep(time.Duration(rand.Int31n(5)) * time.Millisecond) id, err := resource.Resource().TSOAllocator().Allocate(ctx) assert.NoError(t, err) + msg := mock_message.NewMockImmutableMessage(t) + msg.EXPECT().MessageID().Return(walimplstest.NewTestMessageID(int64(id))).Maybe() ts.Ack( - OptMessageID(walimplstest.NewTestMessageID(int64(id))), + OptImmutableMessage(msg), ) }() } @@ -216,9 +219,9 @@ func TestAckManager(t *testing.T) { time.Sleep(time.Duration(rand.Int31n(5)) * time.Millisecond) id, err := resource.Resource().TSOAllocator().Allocate(ctx) assert.NoError(t, err) - ts.Ack( - OptMessageID(walimplstest.NewTestMessageID(int64(id))), - ) + msg := mock_message.NewMockImmutableMessage(t) + msg.EXPECT().MessageID().Return(walimplstest.NewTestMessageID(int64(id))).Maybe() + ts.Ack(OptImmutableMessage(msg)) }(i) } wg.Wait() diff --git a/internal/streamingnode/server/wal/interceptors/timetick/ack/detail.go b/internal/streamingnode/server/wal/interceptors/timetick/ack/detail.go index 96fd24c97b14f..7008ddb804819 100644 --- a/internal/streamingnode/server/wal/interceptors/timetick/ack/detail.go +++ b/internal/streamingnode/server/wal/interceptors/timetick/ack/detail.go @@ -26,7 +26,7 @@ type AckDetail struct { EndTimestamp uint64 // the timestamp when acker is acknowledged. // for avoiding allocation of timestamp failure, the timestamp will use the ack manager last allocated timestamp. LastConfirmedMessageID message.MessageID - MessageID message.MessageID + Message message.ImmutableMessage TxnSession *txn.TxnSession IsSync bool Err error @@ -49,10 +49,10 @@ func OptError(err error) AckOption { } } -// OptMessageID marks the message id for acker. -func OptMessageID(messageID message.MessageID) AckOption { +// OptImmutableMessage marks the acker is done. +func OptImmutableMessage(msg message.ImmutableMessage) AckOption { return func(detail *AckDetail) { - detail.MessageID = messageID + detail.Message = msg } } diff --git a/internal/streamingnode/server/wal/interceptors/timetick/ack/detail_test.go b/internal/streamingnode/server/wal/interceptors/timetick/ack/detail_test.go index f1062ecc0b10f..88a54ec54fe92 100644 --- a/internal/streamingnode/server/wal/interceptors/timetick/ack/detail_test.go +++ b/internal/streamingnode/server/wal/interceptors/timetick/ack/detail_test.go @@ -28,8 +28,9 @@ func TestDetail(t *testing.T) { OptError(errors.New("test"))(ackDetail) assert.Error(t, ackDetail.Err) - OptMessageID(walimplstest.NewTestMessageID(1))(ackDetail) - assert.NotNil(t, ackDetail.MessageID) + msg := mock_message.NewMockImmutableMessage(t) + OptImmutableMessage(msg)(ackDetail) + assert.NotNil(t, ackDetail.Message) OptTxnSession(&txn.TxnSession{})(ackDetail) assert.NotNil(t, ackDetail.TxnSession) diff --git a/internal/streamingnode/server/wal/interceptors/timetick/ack/last_confirmed.go b/internal/streamingnode/server/wal/interceptors/timetick/ack/last_confirmed.go index c43d894a876c6..2f30571ab4fbf 100644 --- a/internal/streamingnode/server/wal/interceptors/timetick/ack/last_confirmed.go +++ b/internal/streamingnode/server/wal/interceptors/timetick/ack/last_confirmed.go @@ -33,7 +33,7 @@ func (m *lastConfirmedManager) AddConfirmedDetails(details sortedDetails, ts uin } m.notDoneTxnMessage.Push(&uncommittedTxnInfo{ session: detail.TxnSession, - messageID: detail.MessageID, + messageID: detail.Message.MessageID(), }) } m.updateLastConfirmedMessageID(ts) diff --git a/internal/streamingnode/server/wal/interceptors/timetick/inspector/inspector.go b/internal/streamingnode/server/wal/interceptors/timetick/inspector/inspector.go index b726748092b00..063490b51360c 100644 --- a/internal/streamingnode/server/wal/interceptors/timetick/inspector/inspector.go +++ b/internal/streamingnode/server/wal/interceptors/timetick/inspector/inspector.go @@ -3,15 +3,17 @@ package inspector import ( "context" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/wab" "github.com/milvus-io/milvus/pkg/streaming/util/types" ) type TimeTickSyncOperator interface { - TimeTickNotifier() *TimeTickNotifier - // Channel returns the pchannel info. Channel() types.PChannelInfo + // WriteAheadBuffer get the related WriteAhead buffer. + WriteAheadBuffer(ctx context.Context) (wab.ROWriteAheadBuffer, error) + // Sync trigger a sync operation, try to send the timetick message into wal. // Sync operation is a blocking operation, and not thread-safe, will only call in one goroutine. Sync(ctx context.Context) diff --git a/internal/streamingnode/server/wal/interceptors/timetick/inspector/notifier.go b/internal/streamingnode/server/wal/interceptors/timetick/inspector/notifier.go index 1ef94e3dceda7..eb3c58430d5e6 100644 --- a/internal/streamingnode/server/wal/interceptors/timetick/inspector/notifier.go +++ b/internal/streamingnode/server/wal/interceptors/timetick/inspector/notifier.go @@ -3,7 +3,6 @@ package inspector import ( "sync" - "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/syncutil" "github.com/milvus-io/milvus/pkg/util/typeutil" @@ -50,81 +49,3 @@ func (n *syncNotifier) Get() typeutil.Set[types.PChannelInfo] { n.cond.L.Unlock() return signal } - -// TimeTickInfo records the information of time tick. -type TimeTickInfo struct { - MessageID message.MessageID // the message id. - TimeTick uint64 // the time tick. - LastConfirmedMessageID message.MessageID // the last confirmed message id. - // The time tick may be updated, without last timetickMessage -} - -// IsZero returns true if the time tick info is zero. -func (t *TimeTickInfo) IsZero() bool { - return t.TimeTick == 0 -} - -// NewTimeTickNotifier creates a new time tick info listener. -func NewTimeTickNotifier() *TimeTickNotifier { - return &TimeTickNotifier{ - cond: syncutil.NewContextCond(&sync.Mutex{}), - info: TimeTickInfo{}, - } -} - -// TimeTickNotifier is a listener for time tick info. -type TimeTickNotifier struct { - cond *syncutil.ContextCond - info TimeTickInfo -} - -// Update only update the time tick info, but not notify the waiter. -func (l *TimeTickNotifier) Update(info TimeTickInfo) { - l.cond.L.Lock() - if l.info.IsZero() || l.info.MessageID.LT(info.MessageID) { - l.info = info - } - l.cond.L.Unlock() -} - -// OnlyUpdateTs only updates the time tick, and notify the waiter. -func (l *TimeTickNotifier) OnlyUpdateTs(timetick uint64) { - l.cond.LockAndBroadcast() - if !l.info.IsZero() && l.info.TimeTick < timetick { - l.info.TimeTick = timetick - } - l.cond.L.Unlock() -} - -// WatchAtMessageID watch the message id. -// If the message id is not equal to the last message id, return nil channel. -// Or if the time tick is less than the last time tick, return channel. -func (l *TimeTickNotifier) WatchAtMessageID(messageID message.MessageID, ts uint64) <-chan struct{} { - l.cond.L.Lock() - // If incoming messageID is less than the producer messageID, - // the consumer can read the new greater messageID from wal, - // so the watch operation is not necessary. - if l.info.IsZero() || messageID.LT(l.info.MessageID) { - l.cond.L.Unlock() - return nil - } - - // messageID may be greater than MessageID in notifier. - // because consuming operation is fast than produce operation. - // so doing a listening here. - if ts < l.info.TimeTick { - ch := make(chan struct{}) - close(ch) - l.cond.L.Unlock() - return ch - } - return l.cond.WaitChan() -} - -// Get gets the time tick info. -func (l *TimeTickNotifier) Get() TimeTickInfo { - l.cond.L.Lock() - info := l.info - l.cond.L.Unlock() - return info -} diff --git a/internal/streamingnode/server/wal/interceptors/timetick/inspector/notifier_test.go b/internal/streamingnode/server/wal/interceptors/timetick/inspector/notifier_test.go index 51a0ddc439bee..51998913e70d2 100644 --- a/internal/streamingnode/server/wal/interceptors/timetick/inspector/notifier_test.go +++ b/internal/streamingnode/server/wal/interceptors/timetick/inspector/notifier_test.go @@ -6,7 +6,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus/pkg/streaming/util/types" - "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" ) func TestSyncNotifier(t *testing.T) { @@ -40,36 +39,3 @@ func shouldBeBlocked(ch <-chan struct{}) { default: } } - -func TestTimeTickNotifier(t *testing.T) { - n := NewTimeTickNotifier() - info := n.Get() - assert.True(t, info.IsZero()) - msgID := walimplstest.NewTestMessageID(1) - assert.Nil(t, n.WatchAtMessageID(msgID, 0)) - n.Update(TimeTickInfo{ - MessageID: msgID, - TimeTick: 2, - LastConfirmedMessageID: walimplstest.NewTestMessageID(0), - }) - - ch := n.WatchAtMessageID(msgID, 0) - assert.NotNil(t, ch) - <-ch // should not block. - - ch = n.WatchAtMessageID(msgID, 2) - assert.NotNil(t, ch) - shouldBeBlocked(ch) // should block. - - n.OnlyUpdateTs(3) - <-ch // should not block. - info = n.Get() - assert.Equal(t, uint64(3), info.TimeTick) - - ch = n.WatchAtMessageID(msgID, 3) - n.Update(TimeTickInfo{ - MessageID: walimplstest.NewTestMessageID(3), - TimeTick: 4, - }) - shouldBeBlocked(ch) -} diff --git a/internal/streamingnode/server/wal/interceptors/timetick/timetick_interceptor.go b/internal/streamingnode/server/wal/interceptors/timetick/timetick_interceptor.go index 15fcf5af44b77..65bb65fd1d34f 100644 --- a/internal/streamingnode/server/wal/interceptors/timetick/timetick_interceptor.go +++ b/internal/streamingnode/server/wal/interceptors/timetick/timetick_interceptor.go @@ -16,7 +16,10 @@ import ( "github.com/milvus-io/milvus/pkg/streaming/util/message" ) -var _ interceptors.InterceptorWithReady = (*timeTickAppendInterceptor)(nil) +var ( + _ interceptors.InterceptorWithReady = (*timeTickAppendInterceptor)(nil) + _ interceptors.InterceptorWithGracefulClose = (*timeTickAppendInterceptor)(nil) +) // timeTickAppendInterceptor is a append interceptor. type timeTickAppendInterceptor struct { @@ -56,7 +59,7 @@ func (impl *timeTickAppendInterceptor) DoAppend(ctx context.Context, msg message return } acker.Ack( - ack.OptMessageID(msgID), + ack.OptImmutableMessage(msg.IntoImmutableMessage(msgID)), ack.OptTxnSession(txnSession), ) }() @@ -201,7 +204,6 @@ func (impl *timeTickAppendInterceptor) appendMsg( if err != nil { return nil, err } - utility.ReplaceAppendResultTimeTick(ctx, msg.TimeTick()) utility.ReplaceAppendResultTxnContext(ctx, msg.TxnContext()) return msgID, nil diff --git a/internal/streamingnode/server/wal/interceptors/timetick/timetick_sync_operator.go b/internal/streamingnode/server/wal/interceptors/timetick/timetick_sync_operator.go index d3bc977212495..37aed02bdb36b 100644 --- a/internal/streamingnode/server/wal/interceptors/timetick/timetick_sync_operator.go +++ b/internal/streamingnode/server/wal/interceptors/timetick/timetick_sync_operator.go @@ -11,6 +11,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick/ack" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick/inspector" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/wab" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/metricsutil" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/utility" "github.com/milvus-io/milvus/pkg/log" @@ -39,7 +40,6 @@ func newTimeTickSyncOperator(param interceptors.InterceptorBuildParam) *timeTick ackManager: nil, ackDetails: ack.NewAckDetails(), sourceID: paramtable.GetNodeID(), - timeTickNotifier: inspector.NewTimeTickNotifier(), metrics: metricsutil.NewTimeTickMetrics(param.WALImpls.Channel().Name), } } @@ -56,20 +56,28 @@ type timeTickSyncOperator struct { ackManager *ack.AckManager // ack manager. ackDetails *ack.AckDetails // all acknowledged details, all acked messages but not sent to wal will be kept here. sourceID int64 // the current node id. - timeTickNotifier *inspector.TimeTickNotifier // used to notify the time tick change. + writeAheadBuffer *wab.WriteAheadBuffer // write ahead buffer. metrics *metricsutil.TimeTickMetrics } +// WriteAheadBuffer returns the write ahead buffer. +func (impl *timeTickSyncOperator) WriteAheadBuffer(ctx context.Context) (wab.ROWriteAheadBuffer, error) { + select { + case <-ctx.Done(): + return nil, ctx.Err() + case <-impl.ready: + } + if impl.writeAheadBuffer == nil { + panic("unreachable write ahead buffer is not ready") + } + return impl.writeAheadBuffer, nil +} + // Channel returns the pchannel info. func (impl *timeTickSyncOperator) Channel() types.PChannelInfo { return impl.pchannel } -// TimeTickNotifier returns the time tick notifier. -func (impl *timeTickSyncOperator) TimeTickNotifier() *inspector.TimeTickNotifier { - return impl.timeTickNotifier -} - // Sync trigger a sync operation. // Sync operation is not thread safe, so call it in a single goroutine. func (impl *timeTickSyncOperator) Sync(ctx context.Context) { @@ -143,7 +151,12 @@ func (impl *timeTickSyncOperator) blockUntilSyncTimeTickReady() error { lastErr = errors.Wrap(err, "allocate timestamp failed") continue } - msgID, err := impl.sendPersistentTsMsg(impl.ctx, ts, nil, underlyingWALImpls.Append) + msg, err := NewTimeTickMsg(ts, nil, impl.sourceID) + if err != nil { + lastErr = errors.Wrap(err, "at build time tick msg") + continue + } + msgID, err := underlyingWALImpls.Append(impl.ctx, msg) if err != nil { lastErr = errors.Wrap(err, "send first timestamp message failed") continue @@ -153,7 +166,13 @@ func (impl *timeTickSyncOperator) blockUntilSyncTimeTickReady() error { impl.logger.Info( "send first time tick success", zap.Uint64("timestamp", ts), - zap.String("messageID", msgID.String())) + zap.Stringer("messageID", msgID)) + impl.writeAheadBuffer = wab.NewWirteAheadBuffer( + impl.logger, + 128*1024*1024, + 30*time.Second, + msg.IntoImmutableMessage(msgID), + ) break } // interceptor is ready now. @@ -202,86 +221,57 @@ func (impl *timeTickSyncOperator) sendTsMsg(ctx context.Context, appender func(c // Construct time tick message. ts := impl.ackDetails.LastAllAcknowledgedTimestamp() lastConfirmedMessageID := impl.ackDetails.EarliestLastConfirmedMessageID() + persist := !impl.ackDetails.IsNoPersistedMessage() - if impl.ackDetails.IsNoPersistedMessage() { - // there's no persisted message, so no need to send persistent time tick message. - return impl.sendNoPersistentTsMsg(ctx, ts, appender) - } - // otherwise, send persistent time tick message. - _, err := impl.sendPersistentTsMsg(ctx, ts, lastConfirmedMessageID, appender) - return err + return impl.sendTsMsgToWAL(ctx, ts, lastConfirmedMessageID, persist, appender) } // sendPersistentTsMsg sends persistent time tick message to wal. -func (impl *timeTickSyncOperator) sendPersistentTsMsg(ctx context.Context, +func (impl *timeTickSyncOperator) sendTsMsgToWAL(ctx context.Context, ts uint64, lastConfirmedMessageID message.MessageID, + persist bool, appender func(ctx context.Context, msg message.MutableMessage) (message.MessageID, error), -) (message.MessageID, error) { +) error { msg, err := NewTimeTickMsg(ts, lastConfirmedMessageID, impl.sourceID) if err != nil { - return nil, errors.Wrap(err, "at build time tick msg") + return errors.Wrap(err, "at build time tick msg") } - // Append it to wal. - msgID, err := appender(ctx, msg) - if err != nil { - return nil, errors.Wrapf(err, - "append time tick msg to wal failed, timestamp: %d, previous message counter: %d", - impl.ackDetails.LastAllAcknowledgedTimestamp(), - impl.ackDetails.Len(), - ) - } - - // metrics updates - impl.metrics.CountPersistentTimeTickSync(ts) - impl.ackDetails.Range(func(detail *ack.AckDetail) bool { - impl.metrics.CountSyncTimeTick(detail.IsSync) - return true - }) - // Ack details has been committed to wal, clear it. - impl.ackDetails.Clear() - // Update last time tick message id and time tick. - impl.timeTickNotifier.Update(inspector.TimeTickInfo{ - MessageID: msgID, - TimeTick: ts, - }) - return msgID, nil -} - -// sendNoPersistentTsMsg sends no persistent time tick message to wal. -func (impl *timeTickSyncOperator) sendNoPersistentTsMsg(ctx context.Context, ts uint64, appender func(ctx context.Context, msg message.MutableMessage) (message.MessageID, error)) error { - msg, err := NewTimeTickMsg(ts, nil, impl.sourceID) - if err != nil { - return errors.Wrap(err, "at build time tick msg when send no persist msg") + if !persist { + // there's no persisted message, so no need to send persistent time tick message. + // With the hint of not persisted message, the underlying wal will not persist it. + // but the interceptors will still be triggered. + ctx = utility.WithNotPersisted(ctx, &utility.NotPersistedHint{ + MessageID: lastConfirmedMessageID, + }) } - // with the hint of not persisted message, the underlying wal will not persist it. - // but the interceptors will still be triggered. - ctx = utility.WithNotPersisted(ctx, &utility.NotPersistedHint{ - MessageID: impl.timeTickNotifier.Get().MessageID, - }) - // Append it to wal. - _, err = appender(ctx, msg) + msgID, err := appender(ctx, msg) if err != nil { return errors.Wrapf(err, - "append no persist time tick msg to wal failed, timestamp: %d, previous message counter: %d", + "append time tick msg to wal failed, timestamp: %d, previous message counter: %d", impl.ackDetails.LastAllAcknowledgedTimestamp(), impl.ackDetails.Len(), ) } - // metrics updates. - impl.metrics.CountMemoryTimeTickSync(ts) + // metrics updates + impl.metrics.CountTimeTickSync(ts, persist) + msgs := make([]message.ImmutableMessage, 0, impl.ackDetails.Len()) impl.ackDetails.Range(func(detail *ack.AckDetail) bool { impl.metrics.CountSyncTimeTick(detail.IsSync) + if !detail.IsSync && detail.Err == nil { + msgs = append(msgs, detail.Message) + } return true }) // Ack details has been committed to wal, clear it. impl.ackDetails.Clear() - // Only update time tick. - impl.timeTickNotifier.OnlyUpdateTs(ts) + tsMsg := msg.IntoImmutableMessage(msgID) + // Add it into write ahead buffer. + impl.writeAheadBuffer.Append(msgs, tsMsg) return nil } diff --git a/internal/streamingnode/server/wal/interceptors/timetick/timetick_sync_operator_test.go b/internal/streamingnode/server/wal/interceptors/timetick/timetick_sync_operator_test.go index da4ee8ee663cf..f5df23ba44764 100644 --- a/internal/streamingnode/server/wal/interceptors/timetick/timetick_sync_operator_test.go +++ b/internal/streamingnode/server/wal/interceptors/timetick/timetick_sync_operator_test.go @@ -12,7 +12,6 @@ import ( "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/internal/streamingnode/server/wal/interceptors/timetick/ack" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/utility" "github.com/milvus-io/milvus/pkg/mocks/streaming/mock_walimpls" "github.com/milvus-io/milvus/pkg/streaming/util/message" @@ -66,40 +65,51 @@ func TestTimeTickSyncOperator(t *testing.T) { ctx := context.Background() ts, err := resource.Resource().TSOAllocator().Allocate(ctx) assert.NoError(t, err) - ch := operator.TimeTickNotifier().WatchAtMessageID(msgID, ts) - shouldBlock(ch) + wb, err := operator.WriteAheadBuffer(ctx) + assert.NoError(t, err) + + ctx, cancel := context.WithTimeout(ctx, 20*time.Millisecond) + defer cancel() + r, err := wb.ReadFromExclusiveTimeTick(ctx, ts) + assert.ErrorIs(t, err, context.DeadlineExceeded) + assert.Nil(t, r) // should not trigger any wal operation, but only update the timetick. operator.Sync(ctx) + r, err = wb.ReadFromExclusiveTimeTick(context.Background(), ts) + assert.NoError(t, err) // should not block because timetick updates. - <-ch + msg, err := r.Next(context.Background()) + assert.NoError(t, err) + assert.NotNil(t, msg) + assert.Greater(t, msg.TimeTick(), ts) // Test alloc a real message but not ack. // because the timetick message id is updated, so the old watcher should be invalidated. - ch = operator.TimeTickNotifier().WatchAtMessageID(msgID, operator.TimeTickNotifier().Get().TimeTick) - shouldBlock(ch) - acker, err := operator.AckManager().Allocate(ctx) - assert.NoError(t, err) - // should block timetick notifier. - ts, _ = resource.Resource().TSOAllocator().Allocate(ctx) - ch = operator.TimeTickNotifier().WatchAtMessageID(walimplstest.NewTestMessageID(2), ts) - shouldBlock(ch) - // sync operation just do nothing, so there's no wal operation triggered. - operator.Sync(ctx) - - // After ack, a wal operation will be trigger. - acker.Ack(ack.OptMessageID(msgID), ack.OptTxnSession(nil)) - l.EXPECT().Append(mock.Anything, mock.Anything).Unset() - l.EXPECT().Append(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, mm message.MutableMessage) (*types.AppendResult, error) { - ts, _ := resource.Resource().TSOAllocator().Allocate(ctx) - return &types.AppendResult{ - MessageID: walimplstest.NewTestMessageID(2), - TimeTick: ts, - }, nil - }) - // should trigger a wal operation. - operator.Sync(ctx) - // ch should still be blocked, because the timetick message id is updated, old message id watch is not notified. - shouldBlock(ch) + // ch = operator.TimeTickNotifier().WatchAtMessageID(msgID, operator.TimeTickNotifier().Get().TimeTick) + // shouldBlock(ch) + // acker, err := operator.AckManager().Allocate(ctx) + // assert.NoError(t, err) + // // should block timetick notifier. + // ts, _ = resource.Resource().TSOAllocator().Allocate(ctx) + // ch = operator.TimeTickNotifier().WatchAtMessageID(walimplstest.NewTestMessageID(2), ts) + // shouldBlock(ch) + // // sync operation just do nothing, so there's no wal operation triggered. + // operator.Sync(ctx) + // + // // After ack, a wal operation will be trigger. + // acker.Ack(ack.OptMessageID(msgID), ack.OptTxnSession(nil)) + // l.EXPECT().Append(mock.Anything, mock.Anything).Unset() + // l.EXPECT().Append(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, mm message.MutableMessage) (*types.AppendResult, error) { + // ts, _ := resource.Resource().TSOAllocator().Allocate(ctx) + // return &types.AppendResult{ + // MessageID: walimplstest.NewTestMessageID(2), + // TimeTick: ts, + // }, nil + // }) + // // should trigger a wal operation. + // operator.Sync(ctx) + // // ch should still be blocked, because the timetick message id is updated, old message id watch is not notified. + // shouldBlock(ch) } func shouldBlock(ch <-chan struct{}) { diff --git a/internal/streamingnode/server/wal/interceptors/wab/pending_queue.go b/internal/streamingnode/server/wal/interceptors/wab/pending_queue.go new file mode 100644 index 0000000000000..26a98ff5448a2 --- /dev/null +++ b/internal/streamingnode/server/wal/interceptors/wab/pending_queue.go @@ -0,0 +1,176 @@ +package wab + +import ( + "io" + "time" + + "github.com/cockroachdb/errors" + + "github.com/milvus-io/milvus/pkg/streaming/util/message" +) + +// ErrEvicted is returned when the expected message has been evicted. +var ErrEvicted = errors.New("message has been evicted") + +// messageWithOffset is a message with an offset as a unique continuous identifier. +type messageWithOffset struct { + Message message.ImmutableMessage + Offset int + Eviction time.Time +} + +// newPendingQueue creates a new pendingQueue with given configuration +func newPendingQueue(capacity int, keepAlive time.Duration, lastConfirmedMessage message.ImmutableMessage) *pendingQueue { + pq := &pendingQueue{ + lastTimeTick: 0, + latestOffset: -1, + buf: make([]messageWithOffset, 0, 10), + size: 0, + capacity: capacity, + keepAlive: keepAlive, + } + pq.Push([]message.ImmutableMessage{lastConfirmedMessage}) + return pq +} + +// pendingQueue is a buffer that stores messages in order of time tick. +// pendingQueue only keep the persisted messages in the buffer. +type pendingQueue struct { + lastTimeTick uint64 + latestOffset int + buf []messageWithOffset + size int + capacity int + keepAlive time.Duration +} + +// Push adds messages to the buffer. +func (q *pendingQueue) Push(msgs []message.ImmutableMessage) { + now := time.Now() + for _, msg := range msgs { + q.pushOne(msg, now) + } + q.evict(now) +} + +// Evict removes messages that have been in the buffer for longer than the keepAlive duration. +func (q *pendingQueue) Evict() { + q.evict(time.Now()) +} + +// CurrentOffset returns the next offset of the buffer. +func (q *pendingQueue) CurrentOffset() int { + return q.latestOffset +} + +// push adds a message to the buffer. +func (q *pendingQueue) pushOne(msg message.ImmutableMessage, now time.Time) { + if msg.Version().EQ(message.VersionOld) { + panic("old message version is not supported") + } + if (msg.MessageType() == message.MessageTypeTimeTick && msg.TimeTick() < q.lastTimeTick) || + (msg.MessageType() != message.MessageTypeTimeTick && msg.TimeTick() <= q.lastTimeTick) { + // only timetick message can be repeated with the last time tick. + panic("message time tick is not in ascending order") + } + q.latestOffset++ + q.buf = append(q.buf, messageWithOffset{ + Offset: q.latestOffset, + Message: msg, + Eviction: now.Add(q.keepAlive), + }) + q.size += msg.EstimateSize() + q.lastTimeTick = msg.TimeTick() +} + +// CreateSnapshotFromOffset creates a snapshot of the buffer from the given offset. +// The continuous slice of messages after [offset, ...] will be returned. +func (q *pendingQueue) CreateSnapshotFromOffset(offset int) ([]messageWithOffset, error) { + if offset > q.latestOffset { + if offset != q.latestOffset+1 { + panic("unreachable: bug here, the offset is not continuous") + } + // If the given version is a version that has not been generated yet, we reach the end of the buffer. + // Return io.EOF to perform a block operation. + return nil, io.EOF + } + if len(q.buf) == 0 || offset < q.buf[0].Offset { + // The expected version is out of range, the expected messages has been evicted. + // So return ErrEvicted to indicate a unrecoverable operation. + return nil, ErrEvicted + } + + // Find the offset of the expected offset in the buffer. + idx := offset - q.buf[0].Offset + return q.makeSnapshot(idx), nil +} + +// CreateSnapshotFromExclusiveTimeTick creates a snapshot of the buffer from the given timetick. +// The coutinous slice of messages after (timeTick, ...] will be returned. +func (q *pendingQueue) CreateSnapshotFromExclusiveTimeTick(timeTick uint64) ([]messageWithOffset, error) { + if timeTick >= q.lastTimeTick { + // If the given timetick is a timetick that has not been generated yet, we reach the end of the buffer. + // Return io.EOF to perform a block operation. + return nil, io.EOF + } + if len(q.buf) == 0 || timeTick < q.buf[0].Message.TimeTick() { + // The expected timetick is out of range, the expected messages may evict. + // So return ErrEvicted to indicate a unrecoverable operation. + return nil, ErrEvicted + } + + // Find the offset of the expected timetick in the buffer. + idx := lowerboundOfMessageList(q.buf, timeTick) + return q.makeSnapshot(idx), nil +} + +// makeSnapshot creates a snapshot of the buffer from the given offset. +func (q *pendingQueue) makeSnapshot(idx int) []messageWithOffset { + snapshot := make([]messageWithOffset, len(q.buf)-idx) // we need a extra position to set a time tick message. + copy(snapshot, q.buf[idx:]) + return snapshot +} + +// evict removes messages that have been in the buffer for longer than the keepAlive duration. +func (q *pendingQueue) evict(now time.Time) { + releaseUntilIdx := -1 + needRelease := 0 + if q.size > q.capacity { + needRelease = q.size - q.capacity + } + for i := 0; i < len(q.buf); i++ { + if q.buf[i].Eviction.Before(now) || needRelease > 0 { + releaseUntilIdx = i + needRelease -= q.buf[i].Message.EstimateSize() + } else { + break + } + } + + preservedIdx := releaseUntilIdx + 1 + if preservedIdx > 0 { + for i := 0; i < preservedIdx; i++ { + // reset the message as zero to release the resource. + q.size -= q.buf[i].Message.EstimateSize() + q.buf[i] = messageWithOffset{} + } + q.buf = q.buf[preservedIdx:] + } +} + +// lowerboundOfMessageList returns the lowerbound of the message list. +func lowerboundOfMessageList(data []messageWithOffset, timetick uint64) int { + // perform a lowerbound search here. + left, right := 0, len(data)-1 + result := -1 + for left <= right { + mid := (left + right) / 2 + if data[mid].Message.TimeTick() > timetick { + result = mid + right = mid - 1 + } else { + left = mid + 1 + } + } + return result +} diff --git a/internal/streamingnode/server/wal/interceptors/wab/pending_queue_test.go b/internal/streamingnode/server/wal/interceptors/wab/pending_queue_test.go new file mode 100644 index 0000000000000..f86ea8ad1603a --- /dev/null +++ b/internal/streamingnode/server/wal/interceptors/wab/pending_queue_test.go @@ -0,0 +1,138 @@ +package wab + +import ( + "io" + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message" + "github.com/milvus-io/milvus/pkg/streaming/util/message" +) + +func TestPendingQueue(t *testing.T) { + pq := newPendingQueue(100, 5*time.Second, newImmutableTimeTickMessage(t, 99)) + snapshot, err := pq.CreateSnapshotFromOffset(0) + assert.NoError(t, err) + assert.Len(t, snapshot, 1) + + snapshot, err = pq.CreateSnapshotFromExclusiveTimeTick(100) + assert.ErrorIs(t, err, io.EOF) + assert.Nil(t, snapshot) + + pq.Push([]message.ImmutableMessage{ + newImmutableMessage(t, 100, 10), + newImmutableMessage(t, 101, 20), + newImmutableMessage(t, 103, 30), + newImmutableMessage(t, 104, 40), + }) + assert.Equal(t, pq.CurrentOffset(), 4) + assert.Len(t, pq.buf, 5) + + snapshot, err = pq.CreateSnapshotFromExclusiveTimeTick(100) + assert.NoError(t, err) + assert.Len(t, snapshot, 3) + assert.Equal(t, snapshot[0].Message.TimeTick(), uint64(101)) + assert.Equal(t, snapshot[2].Message.TimeTick(), uint64(104)) + + snapshot, err = pq.CreateSnapshotFromExclusiveTimeTick(98) + assert.ErrorIs(t, err, ErrEvicted) + assert.Nil(t, snapshot) + + snapshot, err = pq.CreateSnapshotFromExclusiveTimeTick(102) + assert.NoError(t, err) + assert.Len(t, snapshot, 2) + assert.Equal(t, snapshot[0].Message.TimeTick(), uint64(103)) + assert.Equal(t, snapshot[1].Message.TimeTick(), uint64(104)) + + snapshot, err = pq.CreateSnapshotFromExclusiveTimeTick(104) + assert.ErrorIs(t, err, io.EOF) + assert.Nil(t, snapshot) + + snapshot, err = pq.CreateSnapshotFromExclusiveTimeTick(105) + assert.ErrorIs(t, err, io.EOF) + assert.Nil(t, snapshot) + + snapshot, err = pq.CreateSnapshotFromOffset(1) + assert.NoError(t, err) + assert.Len(t, snapshot, 4) + assert.Equal(t, snapshot[0].Message.TimeTick(), uint64(100)) + assert.Equal(t, snapshot[3].Message.TimeTick(), uint64(104)) + + snapshot, err = pq.CreateSnapshotFromOffset(3) + assert.NoError(t, err) + assert.Len(t, snapshot, 2) + assert.Equal(t, snapshot[0].Message.TimeTick(), uint64(103)) + assert.Equal(t, snapshot[1].Message.TimeTick(), uint64(104)) + + snapshot, err = pq.CreateSnapshotFromOffset(4) + assert.NoError(t, err) + assert.Len(t, snapshot, 1) + assert.Equal(t, snapshot[0].Message.TimeTick(), uint64(104)) + + snapshot, err = pq.CreateSnapshotFromOffset(5) + assert.ErrorIs(t, err, io.EOF) + assert.Nil(t, snapshot) + + // push a new item will trigger eviction + snapshot, err = pq.CreateSnapshotFromOffset(1) + assert.NoError(t, err, io.EOF) + assert.Len(t, snapshot, 4) + pq.Push([]message.ImmutableMessage{ + newImmutableMessage(t, 105, 60), + }) + assert.Equal(t, pq.CurrentOffset(), 5) + assert.Len(t, pq.buf, 2) + + // Previous snapshot should be available. + assert.Equal(t, snapshot[0].Message.TimeTick(), uint64(100)) + assert.Equal(t, snapshot[1].Message.TimeTick(), uint64(101)) + assert.Equal(t, snapshot[2].Message.TimeTick(), uint64(103)) + assert.Equal(t, snapshot[3].Message.TimeTick(), uint64(104)) + + // offset 2 should be evcited + snapshot, err = pq.CreateSnapshotFromOffset(3) + assert.ErrorIs(t, err, ErrEvicted) + assert.Nil(t, snapshot) + + // offset 3 should be ok. + snapshot, err = pq.CreateSnapshotFromOffset(4) + assert.NoError(t, err) + assert.Len(t, snapshot, 2) + assert.Equal(t, snapshot[0].Message.TimeTick(), uint64(104)) + assert.Equal(t, snapshot[1].Message.TimeTick(), uint64(105)) + + // Test time based eviction + pq = newPendingQueue(100, 10*time.Millisecond, newImmutableTimeTickMessage(t, 99)) + pq.Push([]message.ImmutableMessage{ + newImmutableMessage(t, 100, 10), + }) + assert.Equal(t, pq.CurrentOffset(), 1) + assert.Len(t, pq.buf, 2) + time.Sleep(20 * time.Millisecond) + pq.Evict() + assert.Len(t, pq.buf, 0) + + assert.Panics(t, func() { + pq.Push([]message.ImmutableMessage{newImmutableMessage(t, 99, 10)}) + }) +} + +func newImmutableMessage(t *testing.T, timetick uint64, estimateSize int) message.ImmutableMessage { + msg := mock_message.NewMockImmutableMessage(t) + msg.EXPECT().TimeTick().Return(timetick).Maybe() + msg.EXPECT().EstimateSize().Return(estimateSize).Maybe() + msg.EXPECT().Version().Return(message.VersionV1).Maybe() + msg.EXPECT().MessageType().Return(message.MessageTypeInsert).Maybe() + return msg +} + +func newImmutableTimeTickMessage(t *testing.T, timetick uint64) message.ImmutableMessage { + msg := mock_message.NewMockImmutableMessage(t) + msg.EXPECT().TimeTick().Return(timetick).Maybe() + msg.EXPECT().EstimateSize().Return(0).Maybe() + msg.EXPECT().MessageType().Return(message.MessageTypeTimeTick).Maybe() + msg.EXPECT().Version().Return(message.VersionV1).Maybe() + return msg +} diff --git a/internal/streamingnode/server/wal/interceptors/wab/reader.go b/internal/streamingnode/server/wal/interceptors/wab/reader.go new file mode 100644 index 0000000000000..6722f98fd8e82 --- /dev/null +++ b/internal/streamingnode/server/wal/interceptors/wab/reader.go @@ -0,0 +1,46 @@ +package wab + +import ( + "context" + + "github.com/milvus-io/milvus/pkg/streaming/util/message" +) + +// WriteAheadBufferReader is used to read messages from WriteAheadBuffer. +type WriteAheadBufferReader struct { + nextOffset int + lastTimeTick uint64 + snapshot []messageWithOffset + underlyingBuf *WriteAheadBuffer +} + +// Next returns the next message in the buffer. +func (r *WriteAheadBufferReader) Next(ctx context.Context) (message.ImmutableMessage, error) { + // Consume snapshot first. + if msg := r.nextFromSnapshot(); msg != nil { + return msg, nil + } + + snapshot, err := r.underlyingBuf.createSnapshotFromOffset(ctx, r.nextOffset, r.lastTimeTick) + if err != nil { + return nil, err + } + r.snapshot = snapshot + return r.nextFromSnapshot(), nil +} + +// nextFromSnapshot returns the next message from the snapshot. +func (r *WriteAheadBufferReader) nextFromSnapshot() message.ImmutableMessage { + if len(r.snapshot) == 0 { + return nil + } + nextMsg := r.snapshot[0] + newNextOffset := nextMsg.Offset + 1 + if newNextOffset < r.nextOffset { + panic("unreachable: next offset should be monotonically increasing") + } + r.nextOffset = newNextOffset + r.lastTimeTick = nextMsg.Message.TimeTick() + r.snapshot = r.snapshot[1:] + return nextMsg.Message +} diff --git a/internal/streamingnode/server/wal/interceptors/wab/write_ahead_buffer.go b/internal/streamingnode/server/wal/interceptors/wab/write_ahead_buffer.go new file mode 100644 index 0000000000000..3505ff463236d --- /dev/null +++ b/internal/streamingnode/server/wal/interceptors/wab/write_ahead_buffer.go @@ -0,0 +1,158 @@ +package wab + +import ( + "context" + "io" + "sync" + "time" + + "github.com/cockroachdb/errors" + + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/util/syncutil" +) + +var _ ROWriteAheadBuffer = (*WriteAheadBuffer)(nil) + +// ROWriteAheadBuffer is the interface of the read-only write-ahead buffer. +type ROWriteAheadBuffer interface { + // ReadFromExclusiveTimeTick reads messages from the buffer from the exclusive time tick. + // Return a reader if the timetick can be consumed from the write-ahead buffer, otherwise return error. + ReadFromExclusiveTimeTick(ctx context.Context, timetick uint64) (*WriteAheadBufferReader, error) +} + +// NewWriteAheadBuffer creates a new WriteAheadBuffer. +func NewWirteAheadBuffer( + logger *log.MLogger, + capacity int, + keepalive time.Duration, + lastConfirmedTimeTickMessage message.ImmutableMessage, +) *WriteAheadBuffer { + return &WriteAheadBuffer{ + logger: logger, + cond: syncutil.NewContextCond(&sync.Mutex{}), + pendingMessages: newPendingQueue(capacity, keepalive, lastConfirmedTimeTickMessage), + lastTimeTickMessage: lastConfirmedTimeTickMessage, + } +} + +// WriteAheadBuffer is a buffer that stores messages in order of time tick. +type WriteAheadBuffer struct { + logger *log.MLogger + cond *syncutil.ContextCond + pendingMessages *pendingQueue // The pending message is always sorted by timetick in monotonic ascending order. + // Only keep the persisted messages in the buffer. + lastTimeTickMessage message.ImmutableMessage +} + +// Append appends a message to the buffer. +func (w *WriteAheadBuffer) Append(msgs []message.ImmutableMessage, tsMsg message.ImmutableMessage) { + w.cond.LockAndBroadcast() + defer w.cond.L.Unlock() + + if tsMsg.MessageType() != message.MessageTypeTimeTick { + panic("the message is not a time tick message") + } + if tsMsg.TimeTick() <= w.lastTimeTickMessage.TimeTick() { + panic("the time tick of the message is less or equal than the last time tick message") + } + if len(msgs) > 0 { + if msgs[0].TimeTick() <= w.lastTimeTickMessage.TimeTick() { + panic("the time tick of the message is less than or equal to the last time tick message") + } + if msgs[len(msgs)-1].TimeTick() > tsMsg.TimeTick() { + panic("the time tick of the message is greater than the time tick message") + } + // if the len(msgs) > 0, the tsMsg is a persisted message. + w.pendingMessages.Push(msgs) + w.pendingMessages.Push([]message.ImmutableMessage{tsMsg}) + } else { + w.pendingMessages.Evict() + } + w.lastTimeTickMessage = tsMsg +} + +// ReadFromExclusiveTimeTick reads messages from the buffer from the exclusive time tick. +func (w *WriteAheadBuffer) ReadFromExclusiveTimeTick(ctx context.Context, timetick uint64) (*WriteAheadBufferReader, error) { + snapshot, nextOffset, err := w.createSnapshotFromTimeTick(ctx, timetick) + if err != nil { + return nil, err + } + return &WriteAheadBufferReader{ + nextOffset: nextOffset, + snapshot: snapshot, + underlyingBuf: w, + }, nil +} + +// createSnapshotFromOffset creates a snapshot of the buffer from the given offset. +func (w *WriteAheadBuffer) createSnapshotFromOffset(ctx context.Context, offset int, timeTick uint64) ([]messageWithOffset, error) { + w.cond.L.Lock() + for { + msgs, err := w.pendingMessages.CreateSnapshotFromOffset(offset) + if err == nil { + w.cond.L.Unlock() + return msgs, nil + } + if !errors.Is(err, io.EOF) { + w.cond.L.Unlock() + return nil, err + } + + // error is eof, which means that the time tick is behind the message buffer. + // check if the last time tick is greater than the given time tick. + // if so, return it to update the timetick. + // lastTimeTickMessage will never be nil if call this api. + if w.lastTimeTickMessage.TimeTick() > timeTick { + msg := messageWithOffset{ + Message: w.lastTimeTickMessage, + Offset: w.pendingMessages.CurrentOffset(), + } + w.cond.L.Unlock() + return []messageWithOffset{msg}, nil + } + // Block until the buffer updates. + if err := w.cond.Wait(ctx); err != nil { + return nil, err + } + } +} + +// createSnapshotFromTimeTick creates a snapshot of the buffer from the given time tick. +func (w *WriteAheadBuffer) createSnapshotFromTimeTick(ctx context.Context, timeTick uint64) ([]messageWithOffset, int, error) { + w.cond.L.Lock() + for { + msgs, err := w.pendingMessages.CreateSnapshotFromExclusiveTimeTick(timeTick) + if err == nil { + w.cond.L.Unlock() + return msgs, msgs[0].Offset, nil + } + if !errors.Is(err, io.EOF) { + w.cond.L.Unlock() + return nil, 0, err + } + + // error is eof, which means that the time tick is behind the message buffer. + // The lastTimeTickMessage should always be greater or equal to the lastTimeTick in the pending queue. + + if w.lastTimeTickMessage.TimeTick() > timeTick { + // check if the last time tick is greater than the given time tick, return it to update the timetick. + msg := messageWithOffset{ + Message: w.lastTimeTickMessage, + Offset: w.pendingMessages.CurrentOffset(), // We add a extra timetick message, so reuse the current offset. + } + w.cond.L.Unlock() + return []messageWithOffset{msg}, msg.Offset, nil + } + if w.lastTimeTickMessage.TimeTick() == timeTick { + offset := w.pendingMessages.CurrentOffset() + 1 + w.cond.L.Unlock() + return nil, offset, nil + } + + if err := w.cond.Wait(ctx); err != nil { + return nil, 0, err + } + } +} diff --git a/internal/streamingnode/server/wal/interceptors/wab/write_ahead_buffer_test.go b/internal/streamingnode/server/wal/interceptors/wab/write_ahead_buffer_test.go new file mode 100644 index 0000000000000..424006b2b5ffe --- /dev/null +++ b/internal/streamingnode/server/wal/interceptors/wab/write_ahead_buffer_test.go @@ -0,0 +1,249 @@ +package wab + +import ( + "context" + "math/rand" + "testing" + "time" + + "github.com/stretchr/testify/assert" + + "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/walimplstest" + "github.com/milvus-io/milvus/pkg/util/syncutil" +) + +func TestWriteAheadBufferWithOnlyTrivialTimeTick(t *testing.T) { + ctx := context.Background() + wb := NewWirteAheadBuffer(log.With(), 5*1024*1024, 30*time.Second, createTimeTickMessage(0)) + + // Test timeout + ctx, cancel := context.WithTimeout(ctx, 1*time.Millisecond) + defer cancel() + r, err := wb.ReadFromExclusiveTimeTick(ctx, 100) + assert.Nil(t, r) + assert.ErrorIs(t, err, context.DeadlineExceeded) + + readErr := syncutil.NewFuture[struct{}]() + expectedLastTimeTick := uint64(10000) + go func() { + r, err := wb.ReadFromExclusiveTimeTick(context.Background(), 100) + assert.NoError(t, err) + assert.NotNil(t, r) + lastTimeTick := uint64(0) + for { + msg, err := r.Next(context.Background()) + assert.NoError(t, err) + assert.NotNil(t, msg) + assert.Greater(t, msg.TimeTick(), lastTimeTick) + lastTimeTick = msg.TimeTick() + if msg.TimeTick() > expectedLastTimeTick { + break + } + } + // Because there's no more message updated, so the Next operation should be blocked forever. + ctx, cancel = context.WithTimeout(ctx, 5*time.Millisecond) + defer cancel() + msg, err := r.Next(ctx) + assert.Nil(t, msg) + assert.ErrorIs(t, err, context.DeadlineExceeded) + readErr.Set(struct{}{}) + }() + + // Current the cache last timetick will be push to 100, + // But we make a exclusive read, so the read operation should be blocked. + wb.Append(nil, createTimeTickMessage(100)) + ctx, cancel = context.WithTimeout(ctx, 5*time.Millisecond) + defer cancel() + _, err = readErr.GetWithContext(ctx) + assert.ErrorIs(t, err, context.DeadlineExceeded) + + nextTimeTick := uint64(100) + for { + nextTimeTick += uint64(rand.Int31n(1000)) + wb.Append(nil, createTimeTickMessage(nextTimeTick)) + if nextTimeTick > expectedLastTimeTick { + break + } + } + readErr.Get() + + r, err = wb.ReadFromExclusiveTimeTick(context.Background(), 0) + assert.NoError(t, err) + msg, err := r.Next(context.Background()) + assert.NoError(t, err) + assert.Equal(t, message.MessageTypeTimeTick, msg.MessageType()) + assert.Equal(t, nextTimeTick, msg.TimeTick()) +} + +func TestWriteAheadBuffer(t *testing.T) { + // Concurrent add message into bufffer and make syncup. + // The reader should never lost any message if no eviction happen. + wb := NewWirteAheadBuffer(log.With(), 5*1024*1024, 30*time.Second, createTimeTickMessage(1)) + expectedLastTimeTick := uint64(10000) + ch := make(chan struct{}) + totalCnt := 0 + go func() { + defer close(ch) + nextTimeTick := uint64(100) + for { + msgs := make([]message.ImmutableMessage, 0) + for i := 0; i < int(rand.Int31n(10))+1; i++ { + nextTimeTick += uint64(rand.Int31n(100) + 1) + msgs = append(msgs, createInsertMessage(nextTimeTick)) + if nextTimeTick > expectedLastTimeTick { + break + } + } + wb.Append(msgs, createTimeTickMessage(msgs[len(msgs)-1].TimeTick())) + totalCnt += (len(msgs) + 1) + if nextTimeTick > expectedLastTimeTick { + break + } + } + }() + r1, err := wb.ReadFromExclusiveTimeTick(context.Background(), 1) + assert.NoError(t, err) + assert.NotNil(t, r1) + lastTimeTick := uint64(0) + timeticks := make([]uint64, 0) + for { + msg, err := r1.Next(context.Background()) + assert.NoError(t, err) + if msg.MessageType() == message.MessageTypeTimeTick { + assert.GreaterOrEqual(t, msg.TimeTick(), lastTimeTick) + } else { + assert.Greater(t, msg.TimeTick(), lastTimeTick) + } + lastTimeTick = msg.TimeTick() + timeticks = append(timeticks, msg.TimeTick()) + if msg.TimeTick() > expectedLastTimeTick { + break + } + } + msg, err := r1.Next(context.Background()) + // There should be a time tick message. + assert.NoError(t, err) + assert.Equal(t, message.MessageTypeTimeTick, msg.MessageType()) + + targetTimeTickIdx := len(timeticks) / 2 + targetTimeTick := timeticks[targetTimeTickIdx] + r2, err := wb.ReadFromExclusiveTimeTick(context.Background(), targetTimeTick) + assert.NoError(t, err) + assert.NotNil(t, r2) + lastTimeTick = uint64(0) + for i := 1; ; i++ { + msg, err := r2.Next(context.Background()) + assert.NoError(t, err) + if msg.MessageType() == message.MessageTypeTimeTick { + assert.GreaterOrEqual(t, msg.TimeTick(), lastTimeTick) + } else { + assert.Greater(t, msg.TimeTick(), lastTimeTick) + } + lastTimeTick = msg.TimeTick() + assert.Equal(t, timeticks[targetTimeTickIdx+i], msg.TimeTick()) + if msg.TimeTick() > expectedLastTimeTick { + break + } + } + msg, err = r2.Next(context.Background()) + // There should be a time tick message. + assert.NoError(t, err) + assert.Equal(t, message.MessageTypeTimeTick, msg.MessageType()) + + rEvicted, err := wb.ReadFromExclusiveTimeTick(context.Background(), 0) + assert.Nil(t, rEvicted) + assert.ErrorIs(t, err, ErrEvicted) + // Read from half of the timetick + <-ch + assert.Equal(t, totalCnt, len(timeticks)) + + ctx, cancel := context.WithTimeout(context.Background(), 20*time.Millisecond) + defer cancel() + _, err = r1.Next(ctx) + assert.ErrorIs(t, err, context.DeadlineExceeded) + ctx, cancel = context.WithTimeout(context.Background(), 20*time.Millisecond) + defer cancel() + _, err = r2.Next(ctx) + assert.ErrorIs(t, err, context.DeadlineExceeded) + wb.Append(nil, createTimeTickMessage(timeticks[len(timeticks)-1]+1)) + msg, err = r1.Next(ctx) + assert.Equal(t, message.MessageTypeTimeTick, msg.MessageType()) + assert.NoError(t, err) + msg, err = r2.Next(ctx) + assert.Equal(t, message.MessageTypeTimeTick, msg.MessageType()) + assert.NoError(t, err) +} + +func TestWriteAheadBufferEviction(t *testing.T) { + wb := NewWirteAheadBuffer(log.With(), 5*1024*1024, 50*time.Millisecond, createTimeTickMessage(0)) + + msgs := make([]message.ImmutableMessage, 0) + for i := 1; i < 100; i++ { + msgs = append(msgs, createInsertMessage(uint64(i))) + } + wb.Append(msgs, createTimeTickMessage(99)) + + // We can read from 0 to 100 messages + r, err := wb.ReadFromExclusiveTimeTick(context.Background(), 0) + assert.NoError(t, err) + assert.NotNil(t, r) + msg, err := r.Next(context.Background()) + assert.NoError(t, err) + assert.Equal(t, msg.TimeTick(), uint64(1)) + + msgs = make([]message.ImmutableMessage, 0) + for i := 100; i < 200; i++ { + msgs = append(msgs, createInsertMessage(uint64(i))) + } + wb.Append(msgs, createTimeTickMessage(199)) + time.Sleep(60 * time.Millisecond) + wb.Append(nil, createTimeTickMessage(200)) + // wait for expiration. + + lastTimeTick := uint64(0) + for { + msg, err := r.Next(context.Background()) + if err != nil { + assert.ErrorIs(t, err, ErrEvicted) + break + } + if msg.MessageType() == message.MessageTypeTimeTick { + assert.GreaterOrEqual(t, msg.TimeTick(), lastTimeTick) + } else { + assert.Greater(t, msg.TimeTick(), lastTimeTick) + } + lastTimeTick = msg.TimeTick() + } + assert.Equal(t, uint64(99), lastTimeTick) +} + +func createTimeTickMessage(timetick uint64) message.ImmutableMessage { + msg, err := message.NewTimeTickMessageBuilderV1(). + WithAllVChannel(). + WithHeader(&message.TimeTickMessageHeader{}). + WithBody(&msgpb.TimeTickMsg{}). + BuildMutable() + if err != nil { + panic(err) + } + return msg.WithTimeTick(timetick).IntoImmutableMessage( + walimplstest.NewTestMessageID(1), + ) +} + +func createInsertMessage(timetick uint64) message.ImmutableMessage { + msg, err := message.NewInsertMessageBuilderV1(). + WithVChannel("vchannel"). + WithHeader(&message.InsertMessageHeader{}). + WithBody(&msgpb.InsertRequest{}). + BuildMutable() + if err != nil { + panic(err) + } + return msg.WithTimeTick(timetick).IntoImmutableMessage( + walimplstest.NewTestMessageID(1), + ) +} diff --git a/internal/streamingnode/server/wal/metricsutil/timetick.go b/internal/streamingnode/server/wal/metricsutil/timetick.go index d0c8bc70d8d6f..4dc085b7c7ddd 100644 --- a/internal/streamingnode/server/wal/metricsutil/timetick.go +++ b/internal/streamingnode/server/wal/metricsutil/timetick.go @@ -82,21 +82,17 @@ func (m *TimeTickMetrics) CountSyncTimeTick(isSync bool) { m.mu.Unlock() } -func (m *TimeTickMetrics) CountMemoryTimeTickSync(ts uint64) { +func (m *TimeTickMetrics) CountTimeTickSync(ts uint64, persist bool) { if !m.mu.LockIfNotClosed() { return } - m.nonPersistentTimeTickSyncCounter.Inc() - m.nonPersistentTimeTickSync.Set(tsoutil.PhysicalTimeSeconds(ts)) - m.mu.Unlock() -} - -func (m *TimeTickMetrics) CountPersistentTimeTickSync(ts uint64) { - if !m.mu.LockIfNotClosed() { - return + if persist { + m.persistentTimeTickSyncCounter.Inc() + m.persistentTimeTickSync.Set(tsoutil.PhysicalTimeSeconds(ts)) + } else { + m.nonPersistentTimeTickSyncCounter.Inc() + m.nonPersistentTimeTickSync.Set(tsoutil.PhysicalTimeSeconds(ts)) } - m.persistentTimeTickSyncCounter.Inc() - m.persistentTimeTickSync.Set(tsoutil.PhysicalTimeSeconds(ts)) m.mu.Unlock() } diff --git a/internal/streamingnode/server/wal/scanner.go b/internal/streamingnode/server/wal/scanner.go index 89ca04460f909..334fe6e9732e4 100644 --- a/internal/streamingnode/server/wal/scanner.go +++ b/internal/streamingnode/server/wal/scanner.go @@ -1,8 +1,6 @@ package wal import ( - "context" - "github.com/cockroachdb/errors" "github.com/milvus-io/milvus/pkg/streaming/util/message" @@ -16,10 +14,12 @@ var ErrUpstreamClosed = errors.New("upstream closed") // ReadOption is the option for reading records from the wal. type ReadOption struct { - VChannel string // vchannel name + VChannel string // vchannel is a optional field to select a vchannel to consume. + // If the vchannel is setup, the message that is not belong to these vchannel will be dropped by scanner. + // Otherwise all message on WAL will be sent. DeliverPolicy options.DeliverPolicy MessageFilter []options.DeliverFilter - MesasgeHandler MessageHandler // message handler for message processing. + MesasgeHandler message.Handler // message handler for message processing. // If the message handler is nil (no redundant operation need to apply), // the default message handler will be used, and the receiver will be returned from Chan. // Otherwise, Chan will panic. @@ -45,27 +45,3 @@ type Scanner interface { // Return the error same with `Error` Close() error } - -type HandleParam struct { - Ctx context.Context - Upstream <-chan message.ImmutableMessage - Message message.ImmutableMessage - TimeTickChan <-chan struct{} -} - -type HandleResult struct { - Incoming message.ImmutableMessage // Not nil if upstream return new message. - MessageHandled bool // True if Message is handled successfully. - TimeTickUpdated bool // True if TimeTickChan is triggered. - Error error // Error is context is canceled. -} - -// MessageHandler is used to handle message read from log. -// TODO: should be removed in future after msgstream is removed. -type MessageHandler interface { - // Handle is the callback for handling message. - Handle(param HandleParam) HandleResult - - // Close is called after all messages are handled or handling is interrupted. - Close() -} diff --git a/internal/streamingnode/server/wal/utility/reorder_buffer.go b/internal/streamingnode/server/wal/utility/reorder_buffer.go index 5df9c4d1b07bb..dca49efecd97d 100644 --- a/internal/streamingnode/server/wal/utility/reorder_buffer.go +++ b/internal/streamingnode/server/wal/utility/reorder_buffer.go @@ -7,8 +7,14 @@ import ( "github.com/milvus-io/milvus/pkg/util/typeutil" ) +var ErrTimeTickVoilation = errors.New("time tick violation") + // ReOrderByTimeTickBuffer is a buffer that stores messages and pops them in order of time tick. type ReOrderByTimeTickBuffer struct { + messageIDs typeutil.Set[string] // After enabling write ahead buffer, we has two stream to consume, + // write ahead buffer works with the timetick order, but the walscannerimpl works with the message order. + // so repeated message may generate when the swithing between the two stream. + // The deduplicate is used to avoid the repeated message. messageHeap typeutil.Heap[message.ImmutableMessage] lastPopTimeTick uint64 bytes int @@ -17,6 +23,7 @@ type ReOrderByTimeTickBuffer struct { // NewReOrderBuffer creates a new ReOrderBuffer. func NewReOrderBuffer() *ReOrderByTimeTickBuffer { return &ReOrderByTimeTickBuffer{ + messageIDs: typeutil.NewSet[string](), messageHeap: typeutil.NewHeap[message.ImmutableMessage](&immutableMessageHeap{}), } } @@ -26,9 +33,14 @@ func (r *ReOrderByTimeTickBuffer) Push(msg message.ImmutableMessage) error { // !!! Drop the unexpected broken timetick rule message. // It will be enabled until the first timetick coming. if msg.TimeTick() < r.lastPopTimeTick { - return errors.Errorf("message time tick is less than last pop time tick: %d", r.lastPopTimeTick) + return errors.Wrapf(ErrTimeTickVoilation, "message time tick is less than last pop time tick: %d", r.lastPopTimeTick) + } + msgID := msg.MessageID().Marshal() + if r.messageIDs.Contain(msgID) { + return errors.Errorf("message is duplicated: %s", msgID) } r.messageHeap.Push(msg) + r.messageIDs.Insert(msgID) r.bytes += msg.EstimateSize() return nil } @@ -39,6 +51,7 @@ func (r *ReOrderByTimeTickBuffer) PopUtilTimeTick(timetick uint64) []message.Imm var res []message.ImmutableMessage for r.messageHeap.Len() > 0 && r.messageHeap.Peek().TimeTick() <= timetick { r.bytes -= r.messageHeap.Peek().EstimateSize() + r.messageIDs.Remove(r.messageHeap.Peek().MessageID().Marshal()) res = append(res, r.messageHeap.Pop()) } r.lastPopTimeTick = timetick diff --git a/internal/streamingnode/server/walmanager/manager_impl_test.go b/internal/streamingnode/server/walmanager/manager_impl_test.go index 1c709b81c05ad..9f38bb1f6743e 100644 --- a/internal/streamingnode/server/walmanager/manager_impl_test.go +++ b/internal/streamingnode/server/walmanager/manager_impl_test.go @@ -8,7 +8,6 @@ import ( "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" @@ -33,12 +32,8 @@ func TestManager(t *testing.T) { fDatacoord := syncutil.NewFuture[internaltypes.DataCoordClient]() fDatacoord.Set(datacoord) - flusher := mock_flusher.NewMockFlusher(t) - flusher.EXPECT().RegisterPChannel(mock.Anything, mock.Anything).Return(nil) - resource.InitForTest( t, - resource.OptFlusher(flusher), resource.OptRootCoordClient(fRootcoord), resource.OptDataCoordClient(fDatacoord), ) diff --git a/internal/streamingnode/server/walmanager/wal_lifetime.go b/internal/streamingnode/server/walmanager/wal_lifetime.go index d4a3059462b1e..969d5989d1ea4 100644 --- a/internal/streamingnode/server/walmanager/wal_lifetime.go +++ b/internal/streamingnode/server/walmanager/wal_lifetime.go @@ -6,7 +6,6 @@ import ( "github.com/cockroachdb/errors" "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" @@ -140,7 +139,6 @@ 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 { - 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. @@ -168,14 +166,6 @@ func (w *walLifetime) doLifetimeChanged(expectedState expectedWALState) { return } logger.Info("open new wal done") - err = resource.Resource().Flusher().RegisterPChannel(w.channel, l) - if err != nil { - logger.Warn("open flusher fail", zap.Error(err)) - w.statePair.SetCurrentState(newUnavailableCurrentState(expectedState.Term(), err)) - // wal is opened, if register flusher failure, we should close the wal. - l.Close() - return - } // -> (expectedTerm,true) w.statePair.SetCurrentState(newAvailableCurrentState(l)) } diff --git a/internal/streamingnode/server/walmanager/wal_lifetime_test.go b/internal/streamingnode/server/walmanager/wal_lifetime_test.go index 32d63f5d75b5b..897e5ada3a934 100644 --- a/internal/streamingnode/server/walmanager/wal_lifetime_test.go +++ b/internal/streamingnode/server/walmanager/wal_lifetime_test.go @@ -8,7 +8,6 @@ import ( "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" @@ -28,13 +27,8 @@ func TestWALLifetime(t *testing.T) { fDatacoord := syncutil.NewFuture[internaltypes.DataCoordClient]() fDatacoord.Set(datacoord) - flusher := mock_flusher.NewMockFlusher(t) - flusher.EXPECT().RegisterPChannel(mock.Anything, mock.Anything).Return(nil) - flusher.EXPECT().UnregisterPChannel(mock.Anything).Return() - resource.InitForTest( t, - resource.OptFlusher(flusher), resource.OptRootCoordClient(fRootcoord), resource.OptDataCoordClient(fDatacoord), ) diff --git a/pkg/proto/streaming.proto b/pkg/proto/streaming.proto index 4539bf34395ce..b6554cc081741 100644 --- a/pkg/proto/streaming.proto +++ b/pkg/proto/streaming.proto @@ -504,3 +504,8 @@ message SegmentAssignmentStat { uint64 binlog_counter = 6; uint64 create_segment_time_tick = 7; // The timetick of create segment message in wal. } + +// The WALCheckpoint that is used to recovery the wal scanner. +message WALCheckpoint { + messages.MessageID messageID = 1; +} \ No newline at end of file diff --git a/pkg/proto/streamingpb/streaming.pb.go b/pkg/proto/streamingpb/streaming.pb.go index 850806e5d41bb..183629866bb4a 100644 --- a/pkg/proto/streamingpb/streaming.pb.go +++ b/pkg/proto/streamingpb/streaming.pb.go @@ -3444,6 +3444,54 @@ func (x *SegmentAssignmentStat) GetCreateSegmentTimeTick() uint64 { return 0 } +// The WALCheckpoint that is used to recovery the wal scanner. +type WALCheckpoint struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + MessageID *messagespb.MessageID `protobuf:"bytes,1,opt,name=messageID,proto3" json:"messageID,omitempty"` +} + +func (x *WALCheckpoint) Reset() { + *x = WALCheckpoint{} + if protoimpl.UnsafeEnabled { + mi := &file_streaming_proto_msgTypes[52] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WALCheckpoint) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WALCheckpoint) ProtoMessage() {} + +func (x *WALCheckpoint) ProtoReflect() protoreflect.Message { + mi := &file_streaming_proto_msgTypes[52] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WALCheckpoint.ProtoReflect.Descriptor instead. +func (*WALCheckpoint) Descriptor() ([]byte, []int) { + return file_streaming_proto_rawDescGZIP(), []int{52} +} + +func (x *WALCheckpoint) GetMessageID() *messagespb.MessageID { + if x != nil { + return x.MessageID + } + return nil +} + var File_streaming_proto protoreflect.FileDescriptor var file_streaming_proto_rawDesc = []byte{ @@ -3920,144 +3968,149 @@ var file_streaming_proto_rawDesc = []byte{ 0x65, 0x61, 0x74, 0x65, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x74, 0x69, 0x63, 0x6b, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x15, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x54, - 0x69, 0x63, 0x6b, 0x2a, 0xc5, 0x01, 0x0a, 0x11, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, - 0x4d, 0x65, 0x74, 0x61, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x50, 0x43, 0x48, - 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, - 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x25, 0x0a, 0x21, 0x50, 0x43, + 0x69, 0x63, 0x6b, 0x22, 0x4f, 0x0a, 0x0d, 0x57, 0x41, 0x4c, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x3e, 0x0a, 0x09, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, + 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x52, 0x09, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x49, 0x44, 0x2a, 0xc5, 0x01, 0x0a, 0x11, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x1b, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, - 0x45, 0x5f, 0x55, 0x4e, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, 0x10, - 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, - 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x49, - 0x4e, 0x47, 0x10, 0x02, 0x12, 0x20, 0x0a, 0x1c, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, - 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x53, 0x53, 0x49, - 0x47, 0x4e, 0x45, 0x44, 0x10, 0x03, 0x12, 0x23, 0x0a, 0x1f, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, - 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, - 0x41, 0x56, 0x41, 0x49, 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x04, 0x2a, 0x77, 0x0a, 0x12, 0x42, - 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x12, 0x20, 0x0a, 0x1c, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54, - 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, - 0x4e, 0x10, 0x00, 0x12, 0x20, 0x0a, 0x1c, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, - 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x50, 0x45, 0x4e, 0x44, - 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, - 0x53, 0x54, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x44, 0x4f, - 0x4e, 0x45, 0x10, 0x02, 0x2a, 0xdc, 0x03, 0x0a, 0x0d, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, - 0x6e, 0x67, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, - 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x4b, 0x10, 0x00, 0x12, 0x24, 0x0a, - 0x20, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, - 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x45, 0x58, 0x49, 0x53, - 0x54, 0x10, 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, - 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x46, 0x45, - 0x4e, 0x43, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1e, 0x0a, 0x1a, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, - 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x48, 0x55, 0x54, - 0x44, 0x4f, 0x57, 0x4e, 0x10, 0x03, 0x12, 0x26, 0x0a, 0x22, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, - 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, - 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x53, 0x45, 0x51, 0x10, 0x04, 0x12, 0x29, - 0x0a, 0x25, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, - 0x5f, 0x55, 0x4e, 0x4d, 0x41, 0x54, 0x43, 0x48, 0x45, 0x44, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, - 0x45, 0x4c, 0x5f, 0x54, 0x45, 0x52, 0x4d, 0x10, 0x05, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x54, 0x52, - 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x47, 0x4e, 0x4f, - 0x52, 0x45, 0x44, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x06, 0x12, - 0x18, 0x0a, 0x14, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, - 0x45, 0x5f, 0x49, 0x4e, 0x4e, 0x45, 0x52, 0x10, 0x07, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x54, 0x52, - 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, - 0x49, 0x4c, 0x44, 0x5f, 0x41, 0x52, 0x47, 0x55, 0x4d, 0x45, 0x4e, 0x54, 0x10, 0x08, 0x12, 0x26, - 0x0a, 0x22, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, - 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x45, 0x58, 0x50, - 0x49, 0x52, 0x45, 0x44, 0x10, 0x09, 0x12, 0x2c, 0x0a, 0x28, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, - 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, 0x44, - 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, 0x41, - 0x54, 0x45, 0x10, 0x0a, 0x12, 0x20, 0x0a, 0x1c, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, - 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e, 0x52, 0x45, 0x43, 0x4f, 0x56, 0x45, 0x52, - 0x41, 0x42, 0x4c, 0x45, 0x10, 0x0b, 0x12, 0x1b, 0x0a, 0x16, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, - 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, - 0x10, 0xe7, 0x07, 0x2a, 0xd5, 0x01, 0x0a, 0x16, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, - 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x24, - 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, - 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, - 0x57, 0x4e, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, - 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, - 0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, - 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, - 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x47, 0x52, 0x4f, 0x57, 0x49, 0x4e, 0x47, 0x10, 0x02, - 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, - 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x45, 0x41, - 0x4c, 0x45, 0x44, 0x10, 0x03, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, + 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x25, 0x0a, 0x21, 0x50, + 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, + 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x49, 0x4e, 0x49, 0x54, 0x49, 0x41, 0x4c, 0x49, 0x5a, 0x45, 0x44, + 0x10, 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4d, + 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, + 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x20, 0x0a, 0x1c, 0x50, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, + 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x41, 0x53, 0x53, + 0x49, 0x47, 0x4e, 0x45, 0x44, 0x10, 0x03, 0x12, 0x23, 0x0a, 0x1f, 0x50, 0x43, 0x48, 0x41, 0x4e, + 0x4e, 0x45, 0x4c, 0x5f, 0x4d, 0x45, 0x54, 0x41, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, + 0x4e, 0x41, 0x56, 0x41, 0x49, 0x4c, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x04, 0x2a, 0x77, 0x0a, 0x12, + 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x20, 0x0a, 0x1c, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, + 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, + 0x57, 0x4e, 0x10, 0x00, 0x12, 0x20, 0x0a, 0x1c, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, + 0x54, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x50, 0x45, 0x4e, + 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x1d, 0x0a, 0x19, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, + 0x41, 0x53, 0x54, 0x5f, 0x54, 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x44, + 0x4f, 0x4e, 0x45, 0x10, 0x02, 0x2a, 0xdc, 0x03, 0x0a, 0x0d, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x54, 0x52, 0x45, 0x41, + 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x4b, 0x10, 0x00, 0x12, 0x24, + 0x0a, 0x20, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, + 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x4e, 0x4f, 0x54, 0x5f, 0x45, 0x58, 0x49, + 0x53, 0x54, 0x10, 0x01, 0x12, 0x21, 0x0a, 0x1d, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, + 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f, 0x46, + 0x45, 0x4e, 0x43, 0x45, 0x44, 0x10, 0x02, 0x12, 0x1e, 0x0a, 0x1a, 0x53, 0x54, 0x52, 0x45, 0x41, + 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x4f, 0x4e, 0x5f, 0x53, 0x48, 0x55, + 0x54, 0x44, 0x4f, 0x57, 0x4e, 0x10, 0x03, 0x12, 0x26, 0x0a, 0x22, 0x53, 0x54, 0x52, 0x45, 0x41, + 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, + 0x44, 0x5f, 0x52, 0x45, 0x51, 0x55, 0x45, 0x53, 0x54, 0x5f, 0x53, 0x45, 0x51, 0x10, 0x04, 0x12, + 0x29, 0x0a, 0x25, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, + 0x45, 0x5f, 0x55, 0x4e, 0x4d, 0x41, 0x54, 0x43, 0x48, 0x45, 0x44, 0x5f, 0x43, 0x48, 0x41, 0x4e, + 0x4e, 0x45, 0x4c, 0x5f, 0x54, 0x45, 0x52, 0x4d, 0x10, 0x05, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x54, + 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x47, 0x4e, + 0x4f, 0x52, 0x45, 0x44, 0x5f, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x06, + 0x12, 0x18, 0x0a, 0x14, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, + 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x4e, 0x45, 0x52, 0x10, 0x07, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x54, + 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, + 0x41, 0x49, 0x4c, 0x44, 0x5f, 0x41, 0x52, 0x47, 0x55, 0x4d, 0x45, 0x4e, 0x54, 0x10, 0x08, 0x12, + 0x26, 0x0a, 0x22, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, + 0x45, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x45, 0x58, + 0x50, 0x49, 0x52, 0x45, 0x44, 0x10, 0x09, 0x12, 0x2c, 0x0a, 0x28, 0x53, 0x54, 0x52, 0x45, 0x41, + 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x49, 0x4e, 0x56, 0x41, 0x4c, 0x49, + 0x44, 0x5f, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x41, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x53, 0x54, + 0x41, 0x54, 0x45, 0x10, 0x0a, 0x12, 0x20, 0x0a, 0x1c, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, + 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e, 0x52, 0x45, 0x43, 0x4f, 0x56, 0x45, + 0x52, 0x41, 0x42, 0x4c, 0x45, 0x10, 0x0b, 0x12, 0x1b, 0x0a, 0x16, 0x53, 0x54, 0x52, 0x45, 0x41, + 0x4d, 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, + 0x4e, 0x10, 0xe7, 0x07, 0x2a, 0xd5, 0x01, 0x0a, 0x16, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, + 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, + 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, + 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, - 0x45, 0x5f, 0x46, 0x4c, 0x55, 0x53, 0x48, 0x45, 0x44, 0x10, 0x04, 0x32, 0x89, 0x01, 0x0a, 0x19, - 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, 0x61, - 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, - 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, - 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, - 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, - 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0x00, 0x32, 0x84, 0x01, 0x0a, 0x1e, 0x53, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, - 0x61, 0x73, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x62, 0x0a, 0x09, 0x42, 0x72, - 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, - 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, - 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0xa5, - 0x01, 0x0a, 0x1f, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6f, 0x72, - 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x12, 0x81, 0x01, 0x0a, 0x12, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, - 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, - 0x6e, 0x67, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, - 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x6d, + 0x45, 0x5f, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x24, 0x0a, 0x20, 0x53, + 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, + 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x47, 0x52, 0x4f, 0x57, 0x49, 0x4e, 0x47, 0x10, + 0x02, 0x12, 0x23, 0x0a, 0x1f, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x41, 0x53, 0x53, + 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x45, + 0x41, 0x4c, 0x45, 0x44, 0x10, 0x03, 0x12, 0x24, 0x0a, 0x20, 0x53, 0x45, 0x47, 0x4d, 0x45, 0x4e, + 0x54, 0x5f, 0x41, 0x53, 0x53, 0x49, 0x47, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x53, 0x54, 0x41, + 0x54, 0x45, 0x5f, 0x46, 0x4c, 0x55, 0x53, 0x48, 0x45, 0x44, 0x10, 0x04, 0x32, 0x89, 0x01, 0x0a, + 0x19, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, + 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, + 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, + 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0x00, 0x32, 0x84, 0x01, 0x0a, 0x1e, 0x53, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x42, 0x72, 0x6f, 0x61, 0x64, + 0x63, 0x61, 0x73, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x62, 0x0a, 0x09, 0x42, + 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, + 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x42, 0x72, 0x6f, 0x61, + 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, + 0xa5, 0x01, 0x0a, 0x1f, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6f, + 0x72, 0x64, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x12, 0x81, 0x01, 0x0a, 0x12, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, + 0x6e, 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, + 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, + 0x74, 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0xe1, 0x01, 0x0a, 0x1b, 0x53, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x72, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x60, 0x0a, 0x07, 0x50, 0x72, 0x6f, 0x64, 0x75, + 0x63, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, + 0x75, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x60, 0x0a, 0x07, 0x43, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, + 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, - 0x44, 0x69, 0x73, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0xe1, 0x01, 0x0a, 0x1b, 0x53, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x48, 0x61, 0x6e, 0x64, 0x6c, 0x65, 0x72, 0x53, - 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x60, 0x0a, 0x07, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, - 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, - 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, - 0x6e, 0x67, 0x2e, 0x50, 0x72, 0x6f, 0x64, 0x75, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x60, 0x0a, 0x07, 0x43, 0x6f, 0x6e, 0x73, - 0x75, 0x6d, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, - 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, - 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0xbe, 0x03, 0x0a, 0x1b, 0x53, + 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0xbe, 0x03, 0x0a, 0x1b, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x81, 0x01, 0x0a, 0x06, + 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x12, 0x39, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x41, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x81, 0x01, 0x0a, 0x06, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x12, 0x39, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, + 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, - 0x67, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x81, 0x01, 0x0a, 0x06, 0x41, - 0x73, 0x73, 0x69, 0x67, 0x6e, 0x12, 0x39, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x67, 0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x96, 0x01, 0x0a, 0x0d, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x40, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, - 0x67, 0x65, 0x72, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x41, 0x73, - 0x73, 0x69, 0x67, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x81, - 0x01, 0x0a, 0x06, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x12, 0x39, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, - 0x6e, 0x67, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, - 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, - 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x96, 0x01, 0x0a, 0x0d, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x12, 0x40, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, 0x53, 0x74, - 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x41, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x2e, - 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x33, 0x5a, 0x31, 0x67, - 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x70, 0x62, - 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x41, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, + 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x4d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x33, 0x5a, 0x31, + 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x70, + 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -4073,7 +4126,7 @@ func file_streaming_proto_rawDescGZIP() []byte { } var file_streaming_proto_enumTypes = make([]protoimpl.EnumInfo, 4) -var file_streaming_proto_msgTypes = make([]protoimpl.MessageInfo, 53) +var file_streaming_proto_msgTypes = make([]protoimpl.MessageInfo, 54) var file_streaming_proto_goTypes = []interface{}{ (PChannelMetaState)(0), // 0: milvus.proto.streaming.PChannelMetaState (BroadcastTaskState)(0), // 1: milvus.proto.streaming.BroadcastTaskState @@ -4131,16 +4184,17 @@ var file_streaming_proto_goTypes = []interface{}{ (*StreamingNodeManagerCollectStatusResponse)(nil), // 53: milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse (*SegmentAssignmentMeta)(nil), // 54: milvus.proto.streaming.SegmentAssignmentMeta (*SegmentAssignmentStat)(nil), // 55: milvus.proto.streaming.SegmentAssignmentStat - nil, // 56: milvus.proto.streaming.BroadcastResponse.ResultsEntry - (*messagespb.Message)(nil), // 57: milvus.proto.messages.Message - (*emptypb.Empty)(nil), // 58: google.protobuf.Empty - (*messagespb.MessageID)(nil), // 59: milvus.proto.messages.MessageID - (messagespb.MessageType)(0), // 60: milvus.proto.messages.MessageType - (*messagespb.TxnContext)(nil), // 61: milvus.proto.messages.TxnContext - (*anypb.Any)(nil), // 62: google.protobuf.Any - (*messagespb.ImmutableMessage)(nil), // 63: milvus.proto.messages.ImmutableMessage - (*milvuspb.GetComponentStatesRequest)(nil), // 64: milvus.proto.milvus.GetComponentStatesRequest - (*milvuspb.ComponentStates)(nil), // 65: milvus.proto.milvus.ComponentStates + (*WALCheckpoint)(nil), // 56: milvus.proto.streaming.WALCheckpoint + nil, // 57: milvus.proto.streaming.BroadcastResponse.ResultsEntry + (*messagespb.Message)(nil), // 58: milvus.proto.messages.Message + (*emptypb.Empty)(nil), // 59: google.protobuf.Empty + (*messagespb.MessageID)(nil), // 60: milvus.proto.messages.MessageID + (messagespb.MessageType)(0), // 61: milvus.proto.messages.MessageType + (*messagespb.TxnContext)(nil), // 62: milvus.proto.messages.TxnContext + (*anypb.Any)(nil), // 63: google.protobuf.Any + (*messagespb.ImmutableMessage)(nil), // 64: milvus.proto.messages.ImmutableMessage + (*milvuspb.GetComponentStatesRequest)(nil), // 65: milvus.proto.milvus.GetComponentStatesRequest + (*milvuspb.ComponentStates)(nil), // 66: milvus.proto.milvus.ComponentStates } var file_streaming_proto_depIdxs = []int32{ 17, // 0: milvus.proto.streaming.PChannelAssignmentLog.node:type_name -> milvus.proto.streaming.StreamingNodeInfo @@ -4148,10 +4202,10 @@ var file_streaming_proto_depIdxs = []int32{ 17, // 2: milvus.proto.streaming.PChannelMeta.node:type_name -> milvus.proto.streaming.StreamingNodeInfo 0, // 3: milvus.proto.streaming.PChannelMeta.state:type_name -> milvus.proto.streaming.PChannelMetaState 5, // 4: milvus.proto.streaming.PChannelMeta.histories:type_name -> milvus.proto.streaming.PChannelAssignmentLog - 57, // 5: milvus.proto.streaming.BroadcastTask.message:type_name -> milvus.proto.messages.Message + 58, // 5: milvus.proto.streaming.BroadcastTask.message:type_name -> milvus.proto.messages.Message 1, // 6: milvus.proto.streaming.BroadcastTask.state:type_name -> milvus.proto.streaming.BroadcastTaskState - 57, // 7: milvus.proto.streaming.BroadcastRequest.message:type_name -> milvus.proto.messages.Message - 56, // 8: milvus.proto.streaming.BroadcastResponse.results:type_name -> milvus.proto.streaming.BroadcastResponse.ResultsEntry + 58, // 7: milvus.proto.streaming.BroadcastRequest.message:type_name -> milvus.proto.messages.Message + 57, // 8: milvus.proto.streaming.BroadcastResponse.results:type_name -> milvus.proto.streaming.BroadcastResponse.ResultsEntry 12, // 9: milvus.proto.streaming.AssignmentDiscoverRequest.report_error:type_name -> milvus.proto.streaming.ReportAssignmentErrorRequest 13, // 10: milvus.proto.streaming.AssignmentDiscoverRequest.close:type_name -> milvus.proto.streaming.CloseAssignmentDiscoverRequest 4, // 11: milvus.proto.streaming.ReportAssignmentErrorRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo @@ -4162,27 +4216,27 @@ var file_streaming_proto_depIdxs = []int32{ 18, // 16: milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion.assignments:type_name -> milvus.proto.streaming.StreamingNodeAssignment 17, // 17: milvus.proto.streaming.StreamingNodeAssignment.node:type_name -> milvus.proto.streaming.StreamingNodeInfo 4, // 18: milvus.proto.streaming.StreamingNodeAssignment.channels:type_name -> milvus.proto.streaming.PChannelInfo - 58, // 19: milvus.proto.streaming.DeliverPolicy.all:type_name -> google.protobuf.Empty - 58, // 20: milvus.proto.streaming.DeliverPolicy.latest:type_name -> google.protobuf.Empty - 59, // 21: milvus.proto.streaming.DeliverPolicy.start_from:type_name -> milvus.proto.messages.MessageID - 59, // 22: milvus.proto.streaming.DeliverPolicy.start_after:type_name -> milvus.proto.messages.MessageID + 59, // 19: milvus.proto.streaming.DeliverPolicy.all:type_name -> google.protobuf.Empty + 59, // 20: milvus.proto.streaming.DeliverPolicy.latest:type_name -> google.protobuf.Empty + 60, // 21: milvus.proto.streaming.DeliverPolicy.start_from:type_name -> milvus.proto.messages.MessageID + 60, // 22: milvus.proto.streaming.DeliverPolicy.start_after:type_name -> milvus.proto.messages.MessageID 21, // 23: milvus.proto.streaming.DeliverFilter.time_tick_gt:type_name -> milvus.proto.streaming.DeliverFilterTimeTickGT 22, // 24: milvus.proto.streaming.DeliverFilter.time_tick_gte:type_name -> milvus.proto.streaming.DeliverFilterTimeTickGTE 23, // 25: milvus.proto.streaming.DeliverFilter.message_type:type_name -> milvus.proto.streaming.DeliverFilterMessageType - 60, // 26: milvus.proto.streaming.DeliverFilterMessageType.message_types:type_name -> milvus.proto.messages.MessageType + 61, // 26: milvus.proto.streaming.DeliverFilterMessageType.message_types:type_name -> milvus.proto.messages.MessageType 2, // 27: milvus.proto.streaming.StreamingError.code:type_name -> milvus.proto.streaming.StreamingCode 27, // 28: milvus.proto.streaming.ProduceRequest.produce:type_name -> milvus.proto.streaming.ProduceMessageRequest 28, // 29: milvus.proto.streaming.ProduceRequest.close:type_name -> milvus.proto.streaming.CloseProducerRequest 4, // 30: milvus.proto.streaming.CreateProducerRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo - 57, // 31: milvus.proto.streaming.ProduceMessageRequest.message:type_name -> milvus.proto.messages.Message + 58, // 31: milvus.proto.streaming.ProduceMessageRequest.message:type_name -> milvus.proto.messages.Message 30, // 32: milvus.proto.streaming.ProduceResponse.create:type_name -> milvus.proto.streaming.CreateProducerResponse 31, // 33: milvus.proto.streaming.ProduceResponse.produce:type_name -> milvus.proto.streaming.ProduceMessageResponse 33, // 34: milvus.proto.streaming.ProduceResponse.close:type_name -> milvus.proto.streaming.CloseProducerResponse 32, // 35: milvus.proto.streaming.ProduceMessageResponse.result:type_name -> milvus.proto.streaming.ProduceMessageResponseResult 24, // 36: milvus.proto.streaming.ProduceMessageResponse.error:type_name -> milvus.proto.streaming.StreamingError - 59, // 37: milvus.proto.streaming.ProduceMessageResponseResult.id:type_name -> milvus.proto.messages.MessageID - 61, // 38: milvus.proto.streaming.ProduceMessageResponseResult.txnContext:type_name -> milvus.proto.messages.TxnContext - 62, // 39: milvus.proto.streaming.ProduceMessageResponseResult.extra:type_name -> google.protobuf.Any + 60, // 37: milvus.proto.streaming.ProduceMessageResponseResult.id:type_name -> milvus.proto.messages.MessageID + 62, // 38: milvus.proto.streaming.ProduceMessageResponseResult.txnContext:type_name -> milvus.proto.messages.TxnContext + 63, // 39: milvus.proto.streaming.ProduceMessageResponseResult.extra:type_name -> google.protobuf.Any 38, // 40: milvus.proto.streaming.ConsumeRequest.create_vchannel_consumer:type_name -> milvus.proto.streaming.CreateVChannelConsumerRequest 37, // 41: milvus.proto.streaming.ConsumeRequest.create_vchannel_consumers:type_name -> milvus.proto.streaming.CreateVChannelConsumersRequest 41, // 42: milvus.proto.streaming.ConsumeRequest.close_vchannel:type_name -> milvus.proto.streaming.CloseVChannelConsumerRequest @@ -4199,34 +4253,35 @@ var file_streaming_proto_depIdxs = []int32{ 39, // 53: milvus.proto.streaming.ConsumeResponse.create_vchannels:type_name -> milvus.proto.streaming.CreateVChannelConsumersResponse 42, // 54: milvus.proto.streaming.ConsumeResponse.close_vchannel:type_name -> milvus.proto.streaming.CloseVChannelConsumerResponse 46, // 55: milvus.proto.streaming.ConsumeResponse.close:type_name -> milvus.proto.streaming.CloseConsumerResponse - 63, // 56: milvus.proto.streaming.ConsumeMessageReponse.message:type_name -> milvus.proto.messages.ImmutableMessage + 64, // 56: milvus.proto.streaming.ConsumeMessageReponse.message:type_name -> milvus.proto.messages.ImmutableMessage 4, // 57: milvus.proto.streaming.StreamingNodeManagerAssignRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo 4, // 58: milvus.proto.streaming.StreamingNodeManagerRemoveRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo 52, // 59: milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse.balance_attributes:type_name -> milvus.proto.streaming.StreamingNodeBalanceAttributes 3, // 60: milvus.proto.streaming.SegmentAssignmentMeta.state:type_name -> milvus.proto.streaming.SegmentAssignmentState 55, // 61: milvus.proto.streaming.SegmentAssignmentMeta.stat:type_name -> milvus.proto.streaming.SegmentAssignmentStat - 32, // 62: milvus.proto.streaming.BroadcastResponse.ResultsEntry.value:type_name -> milvus.proto.streaming.ProduceMessageResponseResult - 64, // 63: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest - 9, // 64: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:input_type -> milvus.proto.streaming.BroadcastRequest - 11, // 65: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:input_type -> milvus.proto.streaming.AssignmentDiscoverRequest - 25, // 66: milvus.proto.streaming.StreamingNodeHandlerService.Produce:input_type -> milvus.proto.streaming.ProduceRequest - 34, // 67: milvus.proto.streaming.StreamingNodeHandlerService.Consume:input_type -> milvus.proto.streaming.ConsumeRequest - 47, // 68: milvus.proto.streaming.StreamingNodeManagerService.Assign:input_type -> milvus.proto.streaming.StreamingNodeManagerAssignRequest - 49, // 69: milvus.proto.streaming.StreamingNodeManagerService.Remove:input_type -> milvus.proto.streaming.StreamingNodeManagerRemoveRequest - 51, // 70: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:input_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusRequest - 65, // 71: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates - 10, // 72: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:output_type -> milvus.proto.streaming.BroadcastResponse - 14, // 73: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:output_type -> milvus.proto.streaming.AssignmentDiscoverResponse - 29, // 74: milvus.proto.streaming.StreamingNodeHandlerService.Produce:output_type -> milvus.proto.streaming.ProduceResponse - 43, // 75: milvus.proto.streaming.StreamingNodeHandlerService.Consume:output_type -> milvus.proto.streaming.ConsumeResponse - 48, // 76: milvus.proto.streaming.StreamingNodeManagerService.Assign:output_type -> milvus.proto.streaming.StreamingNodeManagerAssignResponse - 50, // 77: milvus.proto.streaming.StreamingNodeManagerService.Remove:output_type -> milvus.proto.streaming.StreamingNodeManagerRemoveResponse - 53, // 78: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:output_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse - 71, // [71:79] is the sub-list for method output_type - 63, // [63:71] is the sub-list for method input_type - 63, // [63:63] is the sub-list for extension type_name - 63, // [63:63] is the sub-list for extension extendee - 0, // [0:63] is the sub-list for field type_name + 60, // 62: milvus.proto.streaming.WALCheckpoint.messageID:type_name -> milvus.proto.messages.MessageID + 32, // 63: milvus.proto.streaming.BroadcastResponse.ResultsEntry.value:type_name -> milvus.proto.streaming.ProduceMessageResponseResult + 65, // 64: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 9, // 65: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:input_type -> milvus.proto.streaming.BroadcastRequest + 11, // 66: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:input_type -> milvus.proto.streaming.AssignmentDiscoverRequest + 25, // 67: milvus.proto.streaming.StreamingNodeHandlerService.Produce:input_type -> milvus.proto.streaming.ProduceRequest + 34, // 68: milvus.proto.streaming.StreamingNodeHandlerService.Consume:input_type -> milvus.proto.streaming.ConsumeRequest + 47, // 69: milvus.proto.streaming.StreamingNodeManagerService.Assign:input_type -> milvus.proto.streaming.StreamingNodeManagerAssignRequest + 49, // 70: milvus.proto.streaming.StreamingNodeManagerService.Remove:input_type -> milvus.proto.streaming.StreamingNodeManagerRemoveRequest + 51, // 71: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:input_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusRequest + 66, // 72: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 10, // 73: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:output_type -> milvus.proto.streaming.BroadcastResponse + 14, // 74: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:output_type -> milvus.proto.streaming.AssignmentDiscoverResponse + 29, // 75: milvus.proto.streaming.StreamingNodeHandlerService.Produce:output_type -> milvus.proto.streaming.ProduceResponse + 43, // 76: milvus.proto.streaming.StreamingNodeHandlerService.Consume:output_type -> milvus.proto.streaming.ConsumeResponse + 48, // 77: milvus.proto.streaming.StreamingNodeManagerService.Assign:output_type -> milvus.proto.streaming.StreamingNodeManagerAssignResponse + 50, // 78: milvus.proto.streaming.StreamingNodeManagerService.Remove:output_type -> milvus.proto.streaming.StreamingNodeManagerRemoveResponse + 53, // 79: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:output_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse + 72, // [72:80] is the sub-list for method output_type + 64, // [64:72] is the sub-list for method input_type + 64, // [64:64] is the sub-list for extension type_name + 64, // [64:64] is the sub-list for extension extendee + 0, // [0:64] is the sub-list for field type_name } func init() { file_streaming_proto_init() } @@ -4859,6 +4914,18 @@ func file_streaming_proto_init() { return nil } } + file_streaming_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WALCheckpoint); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } file_streaming_proto_msgTypes[7].OneofWrappers = []interface{}{ (*AssignmentDiscoverRequest_ReportError)(nil), @@ -4916,7 +4983,7 @@ func file_streaming_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_streaming_proto_rawDesc, NumEnums: 4, - NumMessages: 53, + NumMessages: 54, NumExtensions: 0, NumServices: 5, }, diff --git a/pkg/streaming/util/message/adaptor/handler.go b/pkg/streaming/util/message/adaptor/handler.go index 80fd72be0766d..a85faf7a9aa0c 100644 --- a/pkg/streaming/util/message/adaptor/handler.go +++ b/pkg/streaming/util/message/adaptor/handler.go @@ -1,8 +1,6 @@ package adaptor import ( - "context" - "go.uber.org/zap" "github.com/milvus-io/milvus/pkg/log" @@ -11,47 +9,96 @@ import ( "github.com/milvus-io/milvus/pkg/util/typeutil" ) +type ChanMessageHandler chan message.ImmutableMessage + +func (h ChanMessageHandler) Handle(param message.HandleParam) message.HandleResult { + var sendingCh chan message.ImmutableMessage + if param.Message != nil { + sendingCh = h + } + select { + case <-param.Ctx.Done(): + return message.HandleResult{Error: param.Ctx.Err()} + case msg, ok := <-param.Upstream: + if !ok { + panic("unreachable code: upstream should never closed") + } + return message.HandleResult{Incoming: msg} + case sendingCh <- param.Message: + return message.HandleResult{MessageHandled: true} + } +} + +func (d ChanMessageHandler) Close() { + close(d) +} + // NewMsgPackAdaptorHandler create a new message pack adaptor handler. func NewMsgPackAdaptorHandler() *MsgPackAdaptorHandler { return &MsgPackAdaptorHandler{ - base: NewBaseMsgPackAdaptorHandler(), + channel: make(chan *msgstream.MsgPack), + base: NewBaseMsgPackAdaptorHandler(), } } -// MsgPackAdaptorHandler is the handler for message pack. type MsgPackAdaptorHandler struct { - base *BaseMsgPackAdaptorHandler + channel chan *msgstream.MsgPack + base *BaseMsgPackAdaptorHandler } // Chan is the channel for message. func (m *MsgPackAdaptorHandler) Chan() <-chan *msgstream.MsgPack { - return m.base.Channel + return m.channel } // Handle is the callback for handling message. -func (m *MsgPackAdaptorHandler) Handle(ctx context.Context, msg message.ImmutableMessage) (bool, error) { - m.base.GenerateMsgPack(msg) - for m.base.PendingMsgPack.Len() > 0 { +func (m *MsgPackAdaptorHandler) Handle(param message.HandleParam) message.HandleResult { + messageHandled := false + // not handle new message if there are pending msgPack. + if param.Message != nil && m.base.PendingMsgPack.Len() == 0 { + m.base.GenerateMsgPack(param.Message) + messageHandled = true + } + + for { + var sendCh chan<- *msgstream.MsgPack + if m.base.PendingMsgPack.Len() != 0 { + sendCh = m.channel + } + select { - case <-ctx.Done(): - return true, ctx.Err() - case m.base.Channel <- m.base.PendingMsgPack.Next(): + case <-param.Ctx.Done(): + return message.HandleResult{ + MessageHandled: messageHandled, + Error: param.Ctx.Err(), + } + case msg, ok := <-param.Upstream: + if !ok { + panic("unreachable code: upstream should never closed") + } + return message.HandleResult{ + Incoming: msg, + MessageHandled: messageHandled, + } + case sendCh <- m.base.PendingMsgPack.Next(): m.base.PendingMsgPack.UnsafeAdvance() + if m.base.PendingMsgPack.Len() > 0 { + continue + } + return message.HandleResult{MessageHandled: messageHandled} } } - return true, nil } -// Close is the callback for closing message. +// Close closes the handler. func (m *MsgPackAdaptorHandler) Close() { - close(m.base.Channel) + close(m.channel) } // NewBaseMsgPackAdaptorHandler create a new base message pack adaptor handler. func NewBaseMsgPackAdaptorHandler() *BaseMsgPackAdaptorHandler { return &BaseMsgPackAdaptorHandler{ Logger: log.With(), - Channel: make(chan *msgstream.MsgPack), Pendings: make([]message.ImmutableMessage, 0), PendingMsgPack: typeutil.NewMultipartQueue[*msgstream.MsgPack](), } @@ -60,7 +107,6 @@ func NewBaseMsgPackAdaptorHandler() *BaseMsgPackAdaptorHandler { // BaseMsgPackAdaptorHandler is the handler for message pack. type BaseMsgPackAdaptorHandler struct { Logger *log.MLogger - Channel chan *msgstream.MsgPack Pendings []message.ImmutableMessage // pendings hold the vOld message which has same time tick. PendingMsgPack *typeutil.MultipartQueue[*msgstream.MsgPack] // pendingMsgPack hold unsent msgPack. } diff --git a/pkg/streaming/util/message/adaptor/handler_test.go b/pkg/streaming/util/message/adaptor/handler_test.go index 1c5909a079739..d34ece8b1da4c 100644 --- a/pkg/streaming/util/message/adaptor/handler_test.go +++ b/pkg/streaming/util/message/adaptor/handler_test.go @@ -3,167 +3,90 @@ package adaptor import ( "context" "testing" - "time" "github.com/stretchr/testify/assert" - "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/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/rmq" ) func TestMsgPackAdaptorHandler(t *testing.T) { - id := rmq.NewRmqID(1) - + messageID := rmq.NewRmqID(1) + tt := uint64(100) + msg := message.CreateTestInsertMessage( + t, + 1, + 1000, + tt, + messageID, + ) + immutableMsg := msg.IntoImmutableMessage(messageID) + + upstream := make(chan message.ImmutableMessage, 1) + + ctx := context.Background() h := NewMsgPackAdaptorHandler() - insertMsg := message.CreateTestInsertMessage(t, 1, 100, 10, id) - insertImmutableMessage := insertMsg.IntoImmutableMessage(id) - ch := make(chan *msgstream.MsgPack, 1) + done := make(chan struct{}) go func() { - for msgPack := range h.Chan() { - ch <- msgPack + for range h.Chan() { } - close(ch) + close(done) }() - ok, err := h.Handle(context.Background(), insertImmutableMessage) - assert.True(t, ok) - assert.NoError(t, err) - msgPack := <-ch - - assert.Equal(t, uint64(10), msgPack.BeginTs) - assert.Equal(t, uint64(10), msgPack.EndTs) - for _, tsMsg := range msgPack.Msgs { - assert.Equal(t, uint64(10), tsMsg.BeginTs()) - assert.Equal(t, uint64(10), tsMsg.EndTs()) - for _, ts := range tsMsg.(*msgstream.InsertMsg).Timestamps { - assert.Equal(t, uint64(10), ts) - } - } - - deleteMsg, err := message.NewDeleteMessageBuilderV1(). - WithVChannel("vchan1"). - WithHeader(&message.DeleteMessageHeader{ - CollectionId: 1, - }). - WithBody(&msgpb.DeleteRequest{ - Base: &commonpb.MsgBase{ - MsgType: commonpb.MsgType_Delete, - }, - CollectionID: 1, - PartitionID: 1, - Timestamps: []uint64{10}, - }). - BuildMutable() - assert.NoError(t, err) + upstream <- immutableMsg + resp := h.Handle(message.HandleParam{ + Ctx: ctx, + Upstream: upstream, + Message: nil, + }) + assert.Equal(t, resp.Incoming, immutableMsg) + assert.False(t, resp.MessageHandled) + assert.NoError(t, resp.Error) + + resp = h.Handle(message.HandleParam{ + Ctx: ctx, + Upstream: upstream, + Message: resp.Incoming, + }) + assert.NoError(t, resp.Error) + assert.Nil(t, resp.Incoming) + assert.True(t, resp.MessageHandled) + h.Close() - deleteImmutableMsg := deleteMsg. - WithTimeTick(11). - WithLastConfirmedUseMessageID(). - IntoImmutableMessage(id) + <-done +} - ok, err = h.Handle(context.Background(), deleteImmutableMsg) - assert.True(t, ok) - assert.NoError(t, err) - msgPack = <-ch - assert.Equal(t, uint64(11), msgPack.BeginTs) - assert.Equal(t, uint64(11), msgPack.EndTs) - for _, tsMsg := range msgPack.Msgs { - assert.Equal(t, uint64(11), tsMsg.BeginTs()) - assert.Equal(t, uint64(11), tsMsg.EndTs()) - for _, ts := range tsMsg.(*msgstream.DeleteMsg).Timestamps { - assert.Equal(t, uint64(11), ts) +func TestDefaultHandler(t *testing.T) { + h := make(ChanMessageHandler, 1) + done := make(chan struct{}) + go func() { + for range h { } - } - - // Create a txn message - msg, err := message.NewBeginTxnMessageBuilderV2(). - WithVChannel("vchan1"). - WithHeader(&message.BeginTxnMessageHeader{ - KeepaliveMilliseconds: 1000, - }). - WithBody(&message.BeginTxnMessageBody{}). - BuildMutable() - assert.NoError(t, err) - assert.NotNil(t, msg) - - txnCtx := message.TxnContext{ - TxnID: 1, - Keepalive: time.Second, - } - - beginImmutableMsg, err := message.AsImmutableBeginTxnMessageV2(msg.WithTimeTick(9). - WithTxnContext(txnCtx). - WithLastConfirmedUseMessageID(). - IntoImmutableMessage(rmq.NewRmqID(2))) - assert.NoError(t, err) - - msg, err = message.NewCommitTxnMessageBuilderV2(). - WithVChannel("vchan1"). - WithHeader(&message.CommitTxnMessageHeader{}). - WithBody(&message.CommitTxnMessageBody{}). - BuildMutable() - assert.NoError(t, err) - - commitImmutableMsg, err := message.AsImmutableCommitTxnMessageV2(msg.WithTimeTick(12). - WithTxnContext(txnCtx). - WithTxnContext(message.TxnContext{}). - WithLastConfirmedUseMessageID(). - IntoImmutableMessage(rmq.NewRmqID(3))) - assert.NoError(t, err) - - txn, err := message.NewImmutableTxnMessageBuilder(beginImmutableMsg). - Add(insertMsg.WithTxnContext(txnCtx).IntoImmutableMessage(id)). - Add(deleteMsg.WithTxnContext(txnCtx).IntoImmutableMessage(id)). - Build(commitImmutableMsg) - assert.NoError(t, err) - - ok, err = h.Handle(context.Background(), txn) - assert.True(t, ok) - assert.NoError(t, err) - msgPack = <-ch - - assert.Equal(t, uint64(12), msgPack.BeginTs) - assert.Equal(t, uint64(12), msgPack.EndTs) - - // Create flush message - msg, err = message.NewFlushMessageBuilderV2(). - WithVChannel("vchan1"). - WithHeader(&message.FlushMessageHeader{}). - WithBody(&message.FlushMessageBody{}). - BuildMutable() - assert.NoError(t, err) - - flushMsg := msg. - WithTimeTick(13). - WithLastConfirmedUseMessageID(). - IntoImmutableMessage(rmq.NewRmqID(4)) - - ok, err = h.Handle(context.Background(), flushMsg) - assert.True(t, ok) - assert.NoError(t, err) - - msgPack = <-ch + close(done) + }() - assert.Equal(t, uint64(13), msgPack.BeginTs) - assert.Equal(t, uint64(13), msgPack.EndTs) + upstream := make(chan message.ImmutableMessage, 1) + msg := mock_message.NewMockImmutableMessage(t) + upstream <- msg + resp := h.Handle(message.HandleParam{ + Ctx: context.Background(), + Upstream: upstream, + Message: nil, + }) + assert.NotNil(t, resp.Incoming) + assert.NoError(t, resp.Error) + assert.False(t, resp.MessageHandled) + assert.Equal(t, resp.Incoming, msg) + + resp = h.Handle(message.HandleParam{ + Ctx: context.Background(), + Upstream: upstream, + Message: resp.Incoming, + }) + assert.NoError(t, resp.Error) + assert.Nil(t, resp.Incoming) + assert.True(t, resp.MessageHandled) h.Close() - <-ch -} - -func TestMsgPackAdaptorHandlerTimeout(t *testing.T) { - id := rmq.NewRmqID(1) - - insertMsg := message.CreateTestInsertMessage(t, 1, 100, 10, id) - insertImmutableMessage := insertMsg.IntoImmutableMessage(id) - - h := NewMsgPackAdaptorHandler() - ctx, cancel := context.WithCancel(context.Background()) - cancel() - - ok, err := h.Handle(ctx, insertImmutableMessage) - assert.True(t, ok) - assert.ErrorIs(t, err, ctx.Err()) + <-done } diff --git a/pkg/streaming/util/message/builder.go b/pkg/streaming/util/message/builder.go index 527add53795d8..90a4805753539 100644 --- a/pkg/streaming/util/message/builder.go +++ b/pkg/streaming/util/message/builder.go @@ -292,16 +292,16 @@ func newImmutableTxnMesasgeFromWAL( return nil, err } // we don't need to modify the begin message's timetick, but set all the timetick of body messages. - for _, m := range body { - m.(*immutableMessageImpl).overwriteTimeTick(commit.TimeTick()) - m.(*immutableMessageImpl).overwriteLastConfirmedMessageID(commit.LastConfirmedMessageID()) + for idx, m := range body { + body[idx] = m.(*immutableMessageImpl).cloneForTxnBody(commit.TimeTick(), commit.LastConfirmedMessageID()) } - immutableMsg := msg.WithTimeTick(commit.TimeTick()). + + immutableMessage := msg.WithTimeTick(commit.TimeTick()). WithLastConfirmed(commit.LastConfirmedMessageID()). WithTxnContext(*commit.TxnContext()). IntoImmutableMessage(commit.MessageID()) return &immutableTxnMessageImpl{ - immutableMessageImpl: *immutableMsg.(*immutableMessageImpl), + immutableMessageImpl: *immutableMessage.(*immutableMessageImpl), begin: begin, messages: body, commit: commit, diff --git a/pkg/streaming/util/message/message_handler.go b/pkg/streaming/util/message/message_handler.go index c6b6355c6a511..064cab3e9eba1 100644 --- a/pkg/streaming/util/message/message_handler.go +++ b/pkg/streaming/util/message/message_handler.go @@ -1,6 +1,22 @@ package message -import "context" +import ( + "context" +) + +// HandleParam is the parameter for handler. +type HandleParam struct { + Ctx context.Context + Upstream <-chan ImmutableMessage + Message ImmutableMessage +} + +// HandleResult is the result of handler. +type HandleResult struct { + Incoming ImmutableMessage // Not nil if upstream return new message. + MessageHandled bool // True if Message is handled successfully. + Error error // Error is context is canceled. +} // Handler is used to handle message read from log. type Handler interface { @@ -8,29 +24,9 @@ type Handler interface { // Return true if the message is consumed, false if the message is not consumed. // Should return error if and only if ctx is done. // !!! It's a bad implementation for compatibility for msgstream, - // should be removed in the future. - Handle(ctx context.Context, msg ImmutableMessage) (bool, error) + // will be removed in the future. + Handle(param HandleParam) HandleResult // Close is called after all messages are handled or handling is interrupted. Close() } - -var _ Handler = ChanMessageHandler(nil) - -// ChanMessageHandler is a handler just forward the message into a channel. -type ChanMessageHandler chan ImmutableMessage - -// Handle is the callback for handling message. -func (cmh ChanMessageHandler) Handle(ctx context.Context, msg ImmutableMessage) (bool, error) { - select { - case <-ctx.Done(): - return false, ctx.Err() - case cmh <- msg: - return true, nil - } -} - -// Close is called after all messages are handled or handling is interrupted. -func (cmh ChanMessageHandler) Close() { - close(cmh) -} diff --git a/pkg/streaming/util/message/message_handler_test.go b/pkg/streaming/util/message/message_handler_test.go deleted file mode 100644 index 12b02810227b9..0000000000000 --- a/pkg/streaming/util/message/message_handler_test.go +++ /dev/null @@ -1,27 +0,0 @@ -package message - -import ( - "context" - "testing" - - "github.com/stretchr/testify/assert" -) - -func TestMessageHandler(t *testing.T) { - ch := make(chan ImmutableMessage, 1) - h := ChanMessageHandler(ch) - ok, err := h.Handle(context.Background(), nil) - assert.NoError(t, err) - assert.True(t, ok) - - ctx, cancel := context.WithCancel(context.Background()) - cancel() - ok, err = h.Handle(ctx, nil) - assert.ErrorIs(t, err, ctx.Err()) - assert.False(t, ok) - - assert.Nil(t, <-ch) - h.Close() - _, ok = <-ch - assert.False(t, ok) -} diff --git a/pkg/streaming/util/message/message_id.go b/pkg/streaming/util/message/message_id.go index f6864506e907a..69bf61e94e53a 100644 --- a/pkg/streaming/util/message/message_id.go +++ b/pkg/streaming/util/message/message_id.go @@ -1,6 +1,8 @@ package message import ( + "fmt" + "github.com/cockroachdb/errors" "github.com/milvus-io/milvus/pkg/util/typeutil" @@ -24,6 +26,15 @@ func RegisterMessageIDUnmsarshaler(name string, unmarshaler MessageIDUnmarshaler // MessageIDUnmarshaler is the unmarshaler for message id. type MessageIDUnmarshaler = func(b string) (MessageID, error) +// MustUnmarshalMessageID unmarshal the message id, panic if failed. +func MustUnmarshalMessageID(name string, b string) MessageID { + id, err := UnmarshalMessageID(name, b) + if err != nil { + panic(fmt.Sprintf("unmarshal message id failed: %s, wal: %s, bytes: %s", err.Error(), name, b)) + } + return id +} + // UnmsarshalMessageID unmarshal the message id. func UnmarshalMessageID(name string, b string) (MessageID, error) { unmarshaler, ok := messageIDUnmarshaler.Get(name) diff --git a/pkg/streaming/util/message/message_impl.go b/pkg/streaming/util/message/message_impl.go index 25c9bcad9ea8d..df86438caf770 100644 --- a/pkg/streaming/util/message/message_impl.go +++ b/pkg/streaming/util/message/message_impl.go @@ -95,9 +95,14 @@ func (m *messageImpl) WithTxnContext(txnCtx TxnContext) MutableMessage { // IntoImmutableMessage converts current message to immutable message. func (m *messageImpl) IntoImmutableMessage(id MessageID) ImmutableMessage { + // payload and id is always immutable, so we only clone the prop here is ok. + prop := m.properties.Clone() return &immutableMessageImpl{ - messageImpl: *m, - id: id, + id: id, + messageImpl: messageImpl{ + payload: m.payload, + properties: prop, + }, } } @@ -238,6 +243,26 @@ func (m *immutableMessageImpl) LastConfirmedMessageID() MessageID { return id } +// cloneForTxnBody clone the message and update timetick and last confirmed message id. +func (m *immutableMessageImpl) cloneForTxnBody(timetick uint64, LastConfirmedMessageID MessageID) *immutableMessageImpl { + newMsg := m.clone() + newMsg.overwriteTimeTick(timetick) + newMsg.overwriteLastConfirmedMessageID(LastConfirmedMessageID) + return newMsg +} + +// clone clones the current message. +func (m *immutableMessageImpl) clone() *immutableMessageImpl { + // payload and message id is always immutable, so we only clone the prop here is ok. + return &immutableMessageImpl{ + id: m.id, + messageImpl: messageImpl{ + payload: m.payload, + properties: m.properties.Clone(), + }, + } +} + // overwriteTimeTick overwrites the time tick of current message. func (m *immutableMessageImpl) overwriteTimeTick(timetick uint64) { m.properties.Delete(messageTimeTick) diff --git a/pkg/streaming/util/message/properties.go b/pkg/streaming/util/message/properties.go index 3f0d120e32fd4..223f3ee2b3b72 100644 --- a/pkg/streaming/util/message/properties.go +++ b/pkg/streaming/util/message/properties.go @@ -65,6 +65,14 @@ func (prop propertiesImpl) ToRawMap() map[string]string { return map[string]string(prop) } +func (prop propertiesImpl) Clone() propertiesImpl { + cloned := make(map[string]string, len(prop)) + for k, v := range prop { + cloned[k] = v + } + return cloned +} + // EstimateSize returns the estimated size of properties. func (prop propertiesImpl) EstimateSize() int { size := 0 diff --git a/pkg/streaming/util/message/version.go b/pkg/streaming/util/message/version.go index 502f7042f652d..bed46259667aa 100644 --- a/pkg/streaming/util/message/version.go +++ b/pkg/streaming/util/message/version.go @@ -28,3 +28,7 @@ func (v Version) String() string { func (v Version) GT(v2 Version) bool { return v > v2 } + +func (v Version) EQ(v2 Version) bool { + return v == v2 +} diff --git a/pkg/streaming/walimpls/scanner.go b/pkg/streaming/walimpls/scanner.go index 3c416c12eeb84..8bcad0f4d7287 100644 --- a/pkg/streaming/walimpls/scanner.go +++ b/pkg/streaming/walimpls/scanner.go @@ -23,6 +23,8 @@ type ScannerImpls interface { Name() string // Chan returns the channel of message. + // If the scanner is failure, the channel will be closed. + // And an error will be returned by Error(). Chan() <-chan message.ImmutableMessage // Error returns the error of scanner failed. diff --git a/tests/integration/minicluster_v2.go b/tests/integration/minicluster_v2.go index 3cde9a60bb51d..e849ddb4c6893 100644 --- a/tests/integration/minicluster_v2.go +++ b/tests/integration/minicluster_v2.go @@ -365,6 +365,7 @@ func (cluster *MiniClusterV2) Start() error { } if streamingutil.IsStreamingServiceEnabled() { + paramtable.SetLocalComponentEnabled(typeutil.StreamingNodeRole) runComponent(cluster.StreamingNode) }