diff --git a/internal/.mockery.yaml b/internal/.mockery.yaml index 537a0b6f395ec..6a06dff2fc825 100644 --- a/internal/.mockery.yaml +++ b/internal/.mockery.yaml @@ -40,7 +40,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: @@ -72,6 +71,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/flushcommon/pipeline/data_sync_service.go b/internal/flushcommon/pipeline/data_sync_service.go index b9f23475b8f02..4854e670a96b4 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,28 @@ func NewStreamingNodeDataSyncService( func NewDataSyncServiceWithMetaCache(metaCache metacache.MetaCache) *DataSyncService { return &DataSyncService{metacache: metaCache} } + +// NewEmptyStreamingNodeDataSyncService is used to create a new data sync service when incoming create collection message. +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 d64c813cf1fde..d82583ad8ef2b 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](64, 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/flushcommon/util/checkpoint_updater.go b/internal/flushcommon/util/checkpoint_updater.go index 302f44db7dab5..31341e6b560f1 100644 --- a/internal/flushcommon/util/checkpoint_updater.go +++ b/internal/flushcommon/util/checkpoint_updater.go @@ -48,8 +48,9 @@ type ChannelCheckpointUpdater struct { tasks map[string]*channelCPUpdateTask notifyChan chan struct{} - closeCh chan struct{} - closeOnce sync.Once + closeCh chan struct{} + closeOnce sync.Once + updateDoneCallback func(*msgpb.MsgPosition) } func NewChannelCheckpointUpdater(broker broker.Broker) *ChannelCheckpointUpdater { @@ -61,6 +62,17 @@ func NewChannelCheckpointUpdater(broker broker.Broker) *ChannelCheckpointUpdater } } +// NewChannelCheckpointUpdaterWithCallback creates a ChannelCheckpointUpdater with a callback function +func NewChannelCheckpointUpdaterWithCallback(broker broker.Broker, updateDoneCallback func(*msgpb.MsgPosition)) *ChannelCheckpointUpdater { + return &ChannelCheckpointUpdater{ + broker: broker, + tasks: make(map[string]*channelCPUpdateTask), + closeCh: make(chan struct{}), + notifyChan: make(chan struct{}, 1), + updateDoneCallback: updateDoneCallback, + } +} + func (ccu *ChannelCheckpointUpdater) Start() { log.Info("channel checkpoint updater start") ticker := time.NewTicker(paramtable.Get().DataNodeCfg.ChannelCheckpointUpdateTickInSeconds.GetAsDuration(time.Second)) @@ -134,6 +146,9 @@ func (ccu *ChannelCheckpointUpdater) updateCheckpoints(tasks []*channelCPUpdateT for _, task := range tasks { task.callback() finished.Insert(task.pos.GetChannelName(), task) + if ccu.updateDoneCallback != nil { + ccu.updateDoneCallback(task.pos) + } } }(tasks) } diff --git a/internal/metastore/catalog.go b/internal/metastore/catalog.go index e576c205d02c7..488b525c3f39b 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/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/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/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/querycoordv2/observers/replica_observer_test.go b/internal/querycoordv2/observers/replica_observer_test.go index a619cdd06ce82..e0a3e2e20033c 100644 --- a/internal/querycoordv2/observers/replica_observer_test.go +++ b/internal/querycoordv2/observers/replica_observer_test.go @@ -204,8 +204,6 @@ func (suite *ReplicaObserverSuite) TestCheckNodesInReplica() { func (suite *ReplicaObserverSuite) TestCheckSQnodesInReplica() { balancer := mock_balancer.NewMockBalancer(suite.T()) - snmanager.StaticStreamingNodeManager.SetBalancerReady(balancer) - change := make(chan struct{}) balancer.EXPECT().WatchChannelAssignments(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, cb func(typeutil.VersionInt64Pair, []types.PChannelInfoAssigned) error) error { versions := []typeutil.VersionInt64Pair{ @@ -249,6 +247,7 @@ func (suite *ReplicaObserverSuite) TestCheckSQnodesInReplica() { <-ctx.Done() return context.Cause(ctx) }) + snmanager.StaticStreamingNodeManager.SetBalancerReady(balancer) ctx := context.Background() err := suite.meta.CollectionManager.PutCollection(ctx, utils.CreateTestCollection(suite.collectionID, 2)) diff --git a/internal/rootcoord/meta_table.go b/internal/rootcoord/meta_table.go index 64aa18e8a10fc..b4b71947a941e 100644 --- a/internal/rootcoord/meta_table.go +++ b/internal/rootcoord/meta_table.go @@ -925,6 +925,11 @@ func (mt *MetaTable) GetPChannelInfo(ctx context.Context, pchannel string) *root Collections: make([]*rootcoordpb.CollectionInfoOnPChannel, 0), } for _, collInfo := range mt.collID2Meta { + if collInfo.State != pb.CollectionState_CollectionCreated { + // streamingnode, skip non-created collections when recovering + // streamingnode will receive the createCollectionMessage to recover if the collection is creating. + continue + } if idx := lo.IndexOf(collInfo.PhysicalChannelNames, pchannel); idx >= 0 { partitions := make([]*rootcoordpb.PartitionInfoOnPChannel, 0, len(collInfo.Partitions)) for _, part := range collInfo.Partitions { 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 53cd894cf7c45..0000000000000 --- a/internal/streamingnode/server/flusher/flusherimpl/channel_lifetime.go +++ /dev/null @@ -1,172 +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" - "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 := adaptor.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/data_service_wrapper.go b/internal/streamingnode/server/flusher/flusherimpl/data_service_wrapper.go new file mode 100644 index 0000000000000..9919cb0c9efdb --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/data_service_wrapper.go @@ -0,0 +1,54 @@ +package flusherimpl + +import ( + "context" + + "github.com/milvus-io/milvus/internal/flushcommon/pipeline" + "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" +) + +// newDataSyncServiceWrapper creates a new data sync service wrapper. +func newDataSyncServiceWrapper(input chan<- *msgstream.MsgPack, ds *pipeline.DataSyncService) *dataSyncServiceWrapper { + handler := adaptor.NewBaseMsgPackAdaptorHandler() + return &dataSyncServiceWrapper{ + input: input, + handler: handler, + ds: ds, + } +} + +// dataSyncServiceWrapper wraps DataSyncService and related input channel. +type dataSyncServiceWrapper struct { + input chan<- *msgstream.MsgPack + handler *adaptor.BaseMsgPackAdaptorHandler + ds *pipeline.DataSyncService +} + +// Start starts the data sync service. +func (ds *dataSyncServiceWrapper) Start() { + ds.ds.Start() +} + +// HandleMessage handles the incoming message. +func (ds *dataSyncServiceWrapper) 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(): + // The input channel will never get stuck because the data sync service will consume the message continuously. + ds.handler.PendingMsgPack.UnsafeAdvance() + } + } + return nil +} + +// Close close the input channel and gracefully close the data sync service. +func (ds *dataSyncServiceWrapper) Close() { + // The input channel should be closed first, otherwise the flowgraph in datasync service will be blocked. + close(ds.input) + ds.ds.GracefullyClose() +} 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..4c129c804b4f9 --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/flusher_components.go @@ -0,0 +1,247 @@ +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/retry" +) + +// flusherComponents is the components of the flusher. +type flusherComponents struct { + wal wal.WAL + broker broker.Broker + syncMgr syncmgr.SyncManager + wbMgr writebuffer.BufferManager + cpUpdater *util.ChannelCheckpointUpdater + chunkManager storage.ChunkManager + dataServices map[string]*dataSyncServiceWrapper + checkpointManager *pchannelCheckpointManager + logger *log.MLogger +} + +// StartMessageID returns the start message id of the flusher after recovering. +func (impl *flusherComponents) StartMessageID() message.MessageID { + return impl.checkpointManager.StartMessageID() +} + +// WhenCreateCollection handles the create collection message. +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) +} + +// 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)) + } + impl.checkpointManager.DropVChannel(vchannel) +} + +// HandleMessage handles the plain message. +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) +} + +// broadcastToAllDataSyncService broadcasts the message to all data sync services. +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 +} + +// addNewDataSyncService adds a new data sync service to the components when new collection is created. +func (impl *flusherComponents) addNewDataSyncService( + createCollectionMsg message.ImmutableCreateCollectionMessageV1, + input chan<- *msgstream.MsgPack, + ds *pipeline.DataSyncService, +) { + impl.checkpointManager.AddVChannel(createCollectionMsg.VChannel(), createCollectionMsg.LastConfirmedMessageID()) + newDS := newDataSyncServiceWrapper(input, ds) + newDS.Start() + impl.dataServices[createCollectionMsg.VChannel()] = newDS + impl.logger.Info("create data sync service done", zap.String("vchannel", createCollectionMsg.VChannel())) +} + +// Close release all the resources of components. +func (impl *flusherComponents) Close() { + for vchannel, ds := range impl.dataServices { + ds.Close() + impl.logger.Info("data sync service closed for flusher closing", zap.String("vchannel", vchannel)) + } + impl.wbMgr.Stop() + impl.cpUpdater.Close() + impl.syncMgr.Close() + impl.checkpointManager.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]*dataSyncServiceWrapper, len(futures)) + var firstErr error + for vchannel, future := range futures { + ds, err := future.Await() + if err == nil { + dataServices[vchannel] = ds.(*dataSyncServiceWrapper) + continue + } + if firstErr == nil { + firstErr = err + } + } + if firstErr != nil { + return firstErr + } + impl.dataServices = dataServices + for vchannel, ds := range dataServices { + ds.Start() + impl.logger.Info("start data sync service when recovering", zap.String("vchannel", vchannel)) + } + return nil +} + +// buildDataSyncServiceWithRetry builds the data sync service with retry. +func (impl *flusherComponents) buildDataSyncServiceWithRetry(ctx context.Context, recoverInfo *datapb.GetChannelRecoveryInfoResponse) (*dataSyncServiceWrapper, error) { + var ds *dataSyncServiceWrapper + 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 +} + +// buildDataSyncService builds the data sync service with given recovery info. +func (impl *flusherComponents) buildDataSyncService(ctx context.Context, recoverInfo *datapb.GetChannelRecoveryInfoResponse) (*dataSyncServiceWrapper, 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 newDataSyncServiceWrapper(input, ds), 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/pchannel_checkpoint.go b/internal/streamingnode/server/flusher/flusherimpl/pchannel_checkpoint.go new file mode 100644 index 0000000000000..b708806643f5b --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/pchannel_checkpoint.go @@ -0,0 +1,167 @@ +package flusherimpl + +import ( + "context" + "sync" + "time" + + "github.com/cenkalti/backoff/v4" + "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/messagespb" + "github.com/milvus-io/milvus/pkg/proto/streamingpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/util/syncutil" +) + +// recoverPChannelCheckpointManager recovers the pchannel checkpoint manager from the catalog +func recoverPChannelCheckpointManager( + ctx context.Context, + walName string, + pchannel string, + checkpoints map[string]message.MessageID, +) (*pchannelCheckpointManager, error) { + vchannelManager := newVChannelCheckpointManager(checkpoints) + checkpoint, err := resource.Resource().StreamingNodeCatalog().GetConsumeCheckpoint(ctx, pchannel) + if err != nil { + return nil, err + } + var startMessageID message.MessageID + var previous message.MessageID + if checkpoint != nil { + startMessageID = message.MustUnmarshalMessageID(walName, checkpoint.MessageID.Id) + previous = startMessageID + } else { + startMessageID = vchannelManager.MinimumCheckpoint() + } + u := &pchannelCheckpointManager{ + notifier: syncutil.NewAsyncTaskNotifier[struct{}](), + cond: syncutil.NewContextCond(&sync.Mutex{}), + pchannel: pchannel, + vchannelManager: vchannelManager, + startMessageID: startMessageID, + logger: resource.Resource().Logger().With(zap.String("pchannel", pchannel), log.FieldComponent("checkpoint-updater")), + } + go u.background(previous) + return u, nil +} + +// pchannelCheckpointManager is the struct to update the checkpoint of a pchannel +type pchannelCheckpointManager struct { + notifier *syncutil.AsyncTaskNotifier[struct{}] + cond *syncutil.ContextCond + pchannel string + vchannelManager *vchannelCheckpointManager + startMessageID message.MessageID + logger *log.MLogger +} + +// StartMessageID returns the start message checkpoint of current recovery +func (m *pchannelCheckpointManager) StartMessageID() message.MessageID { + return m.startMessageID +} + +// Update updates the checkpoint of a vchannel +func (m *pchannelCheckpointManager) Update(vchannel string, checkpoint message.MessageID) { + m.cond.L.Lock() + defer m.cond.L.Unlock() + + oldMinimum := m.vchannelManager.MinimumCheckpoint() + err := m.vchannelManager.Update(vchannel, checkpoint) + if err != nil { + m.logger.Warn("failed to update vchannel checkpoint", zap.String("vchannel", vchannel), zap.Error(err)) + return + } + if newMinimum := m.vchannelManager.MinimumCheckpoint(); oldMinimum == nil || oldMinimum.LT(newMinimum) { + // if the minimum checkpoint is updated, notify the background goroutine to update the pchannel checkpoint + m.cond.UnsafeBroadcast() + } +} + +// AddVChannel adds a vchannel to the pchannel +func (m *pchannelCheckpointManager) AddVChannel(vchannel string, checkpoint message.MessageID) { + m.cond.LockAndBroadcast() + defer m.cond.L.Unlock() + + if err := m.vchannelManager.Add(vchannel, checkpoint); err != nil { + m.logger.Warn("failed to add vchannel checkpoint", zap.String("vchannel", vchannel), zap.Error(err)) + } + m.logger.Info("add vchannel checkpoint", zap.String("vchannel", vchannel), zap.Stringer("checkpoint", checkpoint)) +} + +// DropVChannel drops a vchannel from the pchannel +func (m *pchannelCheckpointManager) DropVChannel(vchannel string) { + m.cond.LockAndBroadcast() + defer m.cond.L.Unlock() + + if err := m.vchannelManager.Drop(vchannel); err != nil { + m.logger.Warn("failed to drop vchannel checkpoint", zap.String("vchannel", vchannel), zap.Error(err)) + return + } + m.logger.Info("drop vchannel checkpoint", zap.String("vchannel", vchannel)) +} + +func (m *pchannelCheckpointManager) background(previous message.MessageID) { + defer func() { + m.notifier.Finish(struct{}{}) + m.logger.Info("pchannel checkpoint updater is closed") + }() + previousStr := "nil" + if previous != nil { + previousStr = previous.String() + } + m.logger.Info("pchannel checkpoint updater started", zap.String("previous", previousStr)) + + backoff := backoff.NewExponentialBackOff() + backoff.InitialInterval = 100 * time.Millisecond + backoff.MaxInterval = 10 * time.Second + for { + current, err := m.blockUntilCheckpointUpdate(previous) + if err != nil { + return + } + + if previous == nil || previous.LT(current) { + err := resource.Resource().StreamingNodeCatalog().SaveConsumeCheckpoint(m.notifier.Context(), m.pchannel, &streamingpb.WALCheckpoint{ + MessageID: &messagespb.MessageID{Id: current.Marshal()}, + }) + if err != nil { + nextInterval := backoff.NextBackOff() + m.logger.Warn("failed to update pchannel checkpoint", zap.Stringer("checkpoint", current), zap.Duration("nextRetryInterval", nextInterval), zap.Error(err)) + select { + case <-time.After(nextInterval): + continue + case <-m.notifier.Context().Done(): + return + } + } + backoff.Reset() + previous = current + m.logger.Debug("update pchannel checkpoint", zap.Stringer("current", current)) + } + } +} + +// blockUntilCheckpointUpdate blocks until the checkpoint of the pchannel is updated +func (m *pchannelCheckpointManager) blockUntilCheckpointUpdate(previous message.MessageID) (message.MessageID, error) { + m.cond.L.Lock() + // block until following conditions are met: + // there is at least one vchannel, and minimum checkpoint of all vchannels is greater than previous. + // if the previous is nil, block until there is at least one vchannel. + for m.vchannelManager.Len() == 0 || (previous != nil && m.vchannelManager.MinimumCheckpoint().LTE(previous)) { + if err := m.cond.Wait(m.notifier.Context()); err != nil { + return nil, err + } + } + minimum := m.vchannelManager.MinimumCheckpoint() + m.cond.L.Unlock() + return minimum, nil +} + +// Close closes the pchannel checkpoint updater +func (m *pchannelCheckpointManager) Close() { + m.notifier.Cancel() + m.notifier.BlockUntilFinish() +} diff --git a/internal/streamingnode/server/flusher/flusherimpl/pchannel_checkpoint_test.go b/internal/streamingnode/server/flusher/flusherimpl/pchannel_checkpoint_test.go new file mode 100644 index 0000000000000..e75a73bb8f000 --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/pchannel_checkpoint_test.go @@ -0,0 +1,54 @@ +package flusherimpl + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "go.uber.org/atomic" + + "github.com/milvus-io/milvus/internal/mocks/mock_metastore" + "github.com/milvus-io/milvus/internal/streamingnode/server/resource" + "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/walimpls/impls/rmq" +) + +func TestPChannelCheckpointManager(t *testing.T) { + snMeta := mock_metastore.NewMockStreamingNodeCataLog(t) + resource.InitForTest(t, resource.OptStreamingNodeCatalog(snMeta)) + snMeta.EXPECT().GetConsumeCheckpoint(mock.Anything, mock.Anything).Return(&streamingpb.WALCheckpoint{ + MessageID: &messagespb.MessageID{Id: rmq.NewRmqID(0).Marshal()}, + }, nil) + minimumOne := atomic.NewPointer[message.MessageID](nil) + snMeta.EXPECT().SaveConsumeCheckpoint(mock.Anything, mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, pchannel string, ckpt *streamingpb.WALCheckpoint) error { + id, _ := message.UnmarshalMessageID("rocksmq", ckpt.MessageID.Id) + minimumOne.Store(&id) + return nil + }) + + exists, vchannel, minimum := generateRandomExistsMessageID() + p, err := recoverPChannelCheckpointManager(context.Background(), "rocksmq", "test", exists) + assert.True(t, p.StartMessageID().EQ(rmq.NewRmqID(0))) + + assert.NoError(t, err) + assert.NotNil(t, p) + assert.Eventually(t, func() bool { + newMinimum := minimumOne.Load() + return newMinimum != nil && (*newMinimum).EQ(minimum) + }, 10*time.Second, 10*time.Millisecond) + + p.AddVChannel("vchannel-999", rmq.NewRmqID(1000000)) + p.DropVChannel("vchannel-1000") + p.Update(vchannel, rmq.NewRmqID(1000001)) + + assert.Eventually(t, func() bool { + newMinimum := minimumOne.Load() + return !(*newMinimum).EQ(minimum) + }, 10*time.Second, 10*time.Millisecond) + + p.Close() +} diff --git a/internal/streamingnode/server/flusher/flusherimpl/util.go b/internal/streamingnode/server/flusher/flusherimpl/util.go new file mode 100644 index 0000000000000..d2deae3c11e56 --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/util.go @@ -0,0 +1,107 @@ +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/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/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 from datacoord +func (impl *WALFlusherImpl) getRecoveryInfos(ctx context.Context, vchannel []string) (map[string]*datapb.GetChannelRecoveryInfoResponse, map[string]message.MessageID, 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, nil, errors.Wrapf(err, "when get recovery info of vchannel %s", vchannel[i]) + } + messageIDs := make(map[string]message.MessageID, len(recoveryInfos)) + for v, info := range recoveryInfos { + messageIDs[v] = adaptor.MustGetMessageIDFromMQWrapperIDBytes(impl.wal.Get().WALName(), info.GetInfo().GetSeekPosition().GetMsgID()) + } + return recoveryInfos, messageIDs, 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}) + err = merr.CheckRPCCall(resp, err) + if errors.Is(err, merr.ErrChannelNotAvailable) { + impl.logger.Warn("channel not available because of collection dropped", zap.String("vchannel", vchannel), zap.Int("retryCnt", retryCnt)) + return retry.Unrecoverable(errChannelLifetimeUnrecoverable) + } + if 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/vchannel_checkpoint.go b/internal/streamingnode/server/flusher/flusherimpl/vchannel_checkpoint.go new file mode 100644 index 0000000000000..8a8b69cd3855e --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/vchannel_checkpoint.go @@ -0,0 +1,138 @@ +package flusherimpl + +import ( + "container/heap" + + "github.com/cockroachdb/errors" + + "github.com/milvus-io/milvus/pkg/streaming/util/message" +) + +var ( + errVChannelAlreadyExists = errors.New("vchannel already exists") + errVChannelNotFound = errors.New("vchannel not found") + errRollbackCheckpoint = errors.New("rollback a checkpoint is not allow") +) + +// newVChannelCheckpointManager creates a new vchannelCheckpointManager +func newVChannelCheckpointManager(exists map[string]message.MessageID) *vchannelCheckpointManager { + index := make(map[string]*vchannelCheckpoint) + checkpointHeap := make(vchannelCheckpointHeap, 0, len(exists)) + for vchannel, checkpoint := range exists { + index[vchannel] = &vchannelCheckpoint{ + vchannel: vchannel, + checkpoint: checkpoint, + index: len(checkpointHeap), + } + checkpointHeap = append(checkpointHeap, index[vchannel]) + } + heap.Init(&checkpointHeap) + return &vchannelCheckpointManager{ + checkpointHeap: checkpointHeap, + index: index, + } +} + +// vchannelCheckpointManager is the struct to manage the checkpoints of all vchannels at one pchannel +type vchannelCheckpointManager struct { + checkpointHeap vchannelCheckpointHeap + index map[string]*vchannelCheckpoint +} + +// Add adds a vchannel with a checkpoint to the manager +func (m *vchannelCheckpointManager) Add(vchannel string, checkpoint message.MessageID) error { + if _, ok := m.index[vchannel]; ok { + return errVChannelAlreadyExists + } + vc := &vchannelCheckpoint{ + vchannel: vchannel, + checkpoint: checkpoint, + } + heap.Push(&m.checkpointHeap, vc) + m.index[vchannel] = vc + return nil +} + +// Drop removes a vchannel from the manager +func (m *vchannelCheckpointManager) Drop(vchannel string) error { + vc, ok := m.index[vchannel] + if !ok { + return errVChannelNotFound + } + heap.Remove(&m.checkpointHeap, vc.index) + delete(m.index, vchannel) + return nil +} + +// Update updates the checkpoint of a vchannel +func (m *vchannelCheckpointManager) Update(vchannel string, checkpoint message.MessageID) error { + previous, ok := m.index[vchannel] + if !ok { + return errVChannelNotFound + } + if checkpoint.LT(previous.checkpoint) { + return errors.Wrapf(errRollbackCheckpoint, "checkpoint: %s, previous: %s", checkpoint, previous.checkpoint) + } + if checkpoint.EQ(previous.checkpoint) { + return nil + } + m.checkpointHeap.Update(previous, checkpoint) + return nil +} + +// Len returns the number of vchannels +func (m *vchannelCheckpointManager) Len() int { + return len(m.checkpointHeap) +} + +// MinimumCheckpoint returns the minimum checkpoint of all vchannels +func (m *vchannelCheckpointManager) MinimumCheckpoint() message.MessageID { + if len(m.checkpointHeap) == 0 { + return nil + } + return m.checkpointHeap[0].checkpoint +} + +// vchannelCheckpoint is the struct to hold the checkpoint of a vchannel +type vchannelCheckpoint struct { + vchannel string + checkpoint message.MessageID + index int +} + +// A vchannelCheckpointHeap implements heap.Interface and holds Items. +type vchannelCheckpointHeap []*vchannelCheckpoint + +func (pq vchannelCheckpointHeap) Len() int { return len(pq) } + +func (pq vchannelCheckpointHeap) Less(i, j int) bool { + return pq[i].checkpoint.LT(pq[j].checkpoint) +} + +func (pq vchannelCheckpointHeap) Swap(i, j int) { + pq[i], pq[j] = pq[j], pq[i] + pq[i].index = i + pq[j].index = j +} + +func (pq *vchannelCheckpointHeap) Push(x any) { + n := len(*pq) + item := x.(*vchannelCheckpoint) + item.index = n + *pq = append(*pq, item) +} + +func (pq *vchannelCheckpointHeap) Pop() any { + old := *pq + n := len(old) + item := old[n-1] + old[n-1] = nil // don't stop the GC from reclaiming the item eventually + item.index = -1 // for safety + *pq = old[0 : n-1] + return item +} + +func (pq *vchannelCheckpointHeap) Update(item *vchannelCheckpoint, checkpoint message.MessageID) { + item.checkpoint = checkpoint + heap.Fix(pq, item.index) +} diff --git a/internal/streamingnode/server/flusher/flusherimpl/vchannel_checkpoint_test.go b/internal/streamingnode/server/flusher/flusherimpl/vchannel_checkpoint_test.go new file mode 100644 index 0000000000000..3ce2a66ddad78 --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/vchannel_checkpoint_test.go @@ -0,0 +1,66 @@ +package flusherimpl + +import ( + "fmt" + "math/rand" + "testing" + + "github.com/stretchr/testify/assert" + + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/rmq" +) + +func TestVChannelCheckpointManager(t *testing.T) { + exists, vchannel, minimumX := generateRandomExistsMessageID() + m := newVChannelCheckpointManager(exists) + assert.True(t, m.MinimumCheckpoint().EQ(minimumX)) + + err := m.Add("vchannel-999", rmq.NewRmqID(1000000)) + assert.Error(t, err) + assert.True(t, m.MinimumCheckpoint().EQ(minimumX)) + + err = m.Drop("vchannel-1000") + assert.Error(t, err) + assert.True(t, m.MinimumCheckpoint().EQ(minimumX)) + + err = m.Update("vchannel-1000", rmq.NewRmqID(1000001)) + assert.Error(t, err) + assert.True(t, m.MinimumCheckpoint().EQ(minimumX)) + + err = m.Add("vchannel-1000", rmq.NewRmqID(1000001)) + assert.NoError(t, err) + assert.True(t, m.MinimumCheckpoint().EQ(minimumX)) + + err = m.Update(vchannel, rmq.NewRmqID(1000001)) + assert.NoError(t, err) + assert.False(t, m.MinimumCheckpoint().EQ(minimumX)) + + err = m.Update(vchannel, minimumX) + assert.Error(t, err) + + err = m.Drop("vchannel-501") + assert.NoError(t, err) + for i := 0; i < 1001; i++ { + m.Drop(fmt.Sprintf("vchannel-%d", i)) + } + assert.Len(t, m.index, 0) + assert.Len(t, m.checkpointHeap, 0) + assert.Equal(t, m.Len(), 0) + assert.Nil(t, m.MinimumCheckpoint()) +} + +func generateRandomExistsMessageID() (map[string]message.MessageID, string, message.MessageID) { + minimumX := int64(10000000) + var vchannel string + exists := make(map[string]message.MessageID) + for i := 0; i < 1000; i++ { + x := rand.Int63n(999999) + 2 + exists[fmt.Sprintf("vchannel-%d", i)] = rmq.NewRmqID(x) + if x < minimumX { + minimumX = x + vchannel = fmt.Sprintf("vchannel-%d", i) + } + } + return exists, vchannel, rmq.NewRmqID(minimumX) +} 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..d21167db9d9ee --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/wal_flusher.go @@ -0,0 +1,203 @@ +package flusherimpl + +import ( + "context" + + "github.com/cockroachdb/errors" + "go.uber.org/zap" + + "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + "github.com/milvus-io/milvus/internal/flushcommon/broker" + "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" + "github.com/milvus-io/milvus/internal/flushcommon/util" + "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" + "github.com/milvus-io/milvus/internal/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/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/paramtable" + "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 + } + } + } +} + +// Close closes the wal flusher and release all related resources for it. +func (impl *WALFlusherImpl) Close() { + impl.notifier.Cancel() + impl.notifier.BlockUntilFinish() +} + +// 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, checkpoints, 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))) + + 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") + + // build all components. + broker := broker.NewCoordBroker(dc, paramtable.GetNodeID()) + chunkManager := resource.Resource().ChunkManager() + syncMgr := syncmgr.NewSyncManager(chunkManager) + wbMgr := writebuffer.NewManager(syncMgr) + wbMgr.Start() + + pm, err := recoverPChannelCheckpointManager(ctx, l.WALName(), l.Channel().Name, checkpoints) + if err != nil { + impl.logger.Warn("recover pchannel checkpoint manager failure", zap.Error(err)) + return nil, err + } + cpUpdater := util.NewChannelCheckpointUpdaterWithCallback(broker, func(mp *msgpb.MsgPosition) { + // After vchannel checkpoint updated, notify the pchannel checkpoint manager to work. + pm.Update(mp.ChannelName, adaptor.MustGetMessageIDFromMQWrapperIDBytes(l.WALName(), mp.MsgID)) + }) + go cpUpdater.Start() + + fc := &flusherComponents{ + wal: l, + broker: broker, + syncMgr: syncMgr, + wbMgr: wbMgr, + cpUpdater: cpUpdater, + chunkManager: chunkManager, + dataServices: make(map[string]*dataSyncServiceWrapper), + checkpointManager: pm, + 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 +} + +// generateScanner create a new scanner for the wal. +func (impl *WALFlusherImpl) generateScanner(ctx context.Context, l wal.WAL) (wal.Scanner, error) { + handler := make(adaptor.ChanMessageHandler, 64) + readOpt := wal.ReadOption{ + VChannel: "", // We need consume all message from wal. + MesasgeHandler: handler, + DeliverPolicy: options.DeliverPolicyAll(), + } + if startMessageID := impl.flusherComponents.StartMessageID(); startMessageID != nil { + impl.logger.Info("wal start to scan from minimum checkpoint", zap.Stringer("startMessageID", startMessageID)) + readOpt.DeliverPolicy = options.DeliverPolicyStartAfter(startMessageID) + } + impl.logger.Info("wal start to scan from the beginning") + return l.Read(ctx, readOpt) +} + +// dispatch dispatches the message to the related handler for flusher components. +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. + // TODO: Current drop collection message will be handled by the underlying data sync service. + defer impl.flusherComponents.WhenDropCollection(msg.VChannel()) + } + return impl.flusherComponents.HandleMessage(impl.notifier.Context(), msg) +} diff --git a/internal/streamingnode/server/flusher/flusherimpl/wal_flusher_test.go b/internal/streamingnode/server/flusher/flusherimpl/wal_flusher_test.go new file mode 100644 index 0000000000000..bf2fe9d7addd9 --- /dev/null +++ b/internal/streamingnode/server/flusher/flusherimpl/wal_flusher_test.go @@ -0,0 +1,161 @@ +package flusherimpl + +import ( + "context" + "os" + "testing" + "time" + + "github.com/stretchr/testify/mock" + "go.uber.org/atomic" + "google.golang.org/grpc" + + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/mocks" + "github.com/milvus-io/milvus/internal/mocks/mock_metastore" + "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_wal" + "github.com/milvus-io/milvus/internal/streamingnode/server/resource" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors" + internaltypes "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/streamingutil" + "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/mocks/streaming/mock_walimpls" + "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/types" + "github.com/milvus-io/milvus/pkg/streaming/walimpls/impls/rmq" + "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 TestMain(m *testing.M) { + paramtable.Init() + if code := m.Run(); code != 0 { + os.Exit(code) + } +} + +func TestWALFlusher(t *testing.T) { + streamingutil.SetStreamingServiceEnabled() + defer streamingutil.UnsetStreamingServiceEnabled() + + rootCoord := mocks.NewMockRootCoordClient(t) + rootCoord.EXPECT().GetPChannelInfo(mock.Anything, mock.Anything).Return(&rootcoordpb.GetPChannelInfoResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + Collections: []*rootcoordpb.CollectionInfoOnPChannel{ + { + CollectionId: 100, + Vchannel: "vchannel-1", + }, + { + CollectionId: 100, + Vchannel: "vchannel-2", + }, + }, + }, nil) + snMeta := mock_metastore.NewMockStreamingNodeCataLog(t) + snMeta.EXPECT().GetConsumeCheckpoint(mock.Anything, mock.Anything).Return(nil, nil) + snMeta.EXPECT().SaveConsumeCheckpoint(mock.Anything, mock.Anything, mock.Anything).Return(nil) + + datacoord := newMockDatacoord(t, false) + fDatacoord := syncutil.NewFuture[internaltypes.DataCoordClient]() + fDatacoord.Set(datacoord) + fRootCoord := syncutil.NewFuture[internaltypes.RootCoordClient]() + fRootCoord.Set(rootCoord) + resource.InitForTest( + t, + resource.OptDataCoordClient(fDatacoord), + resource.OptRootCoordClient(fRootCoord), + resource.OptStreamingNodeCatalog(snMeta), + ) + walImpl := mock_walimpls.NewMockWALImpls(t) + walImpl.EXPECT().Channel().Return(types.PChannelInfo{Name: "pchannel"}) + + l := newMockWAL(t, false) + param := interceptors.InterceptorBuildParam{ + WALImpls: walImpl, + WAL: syncutil.NewFuture[wal.WAL](), + } + param.WAL.Set(l) + flusher := RecoverWALFlusher(param) + time.Sleep(5 * time.Second) + flusher.Close() +} + +func newMockDatacoord(t *testing.T, maybe bool) *mocks.MockDataCoordClient { + datacoord := mocks.NewMockDataCoordClient(t) + failureCnt := atomic.NewInt32(2) + datacoord.EXPECT().DropVirtualChannel(mock.Anything, mock.Anything).Return(&datapb.DropVirtualChannelResponse{ + Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, + }, nil) + 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.ErrChannelNotAvailable), + }, 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, DataType: schemapb.DataType_Int64}, + {FieldID: 101, Name: "Vector", DataType: schemapb.DataType_FloatVector}, + }, + }, + }, nil + }) + if maybe { + expect.Maybe() + } + return datacoord +} + +func newMockWAL(t *testing.T, maybe bool) *mock_wal.MockWAL { + w := mock_wal.NewMockWAL(t) + walName := w.EXPECT().WALName().Return("rocksmq") + if maybe { + walName.Maybe() + } + w.EXPECT().Channel().Return(types.PChannelInfo{Name: "pchannel"}).Maybe() + 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) + ch := make(chan message.ImmutableMessage, 4) + msg := message.CreateTestCreateCollectionMessage(t, 2, 100, rmq.NewRmqID(100)) + ch <- msg.IntoImmutableMessage(rmq.NewRmqID(105)) + msg = message.CreateTestCreateSegmentMessage(t, 2, 101, rmq.NewRmqID(101)) + ch <- msg.IntoImmutableMessage(rmq.NewRmqID(106)) + msg = message.CreateTestTimeTickSyncMessage(t, 2, 102, rmq.NewRmqID(101)) + ch <- msg.IntoImmutableMessage(rmq.NewRmqID(107)) + msg = message.CreateTestDropCollectionMessage(t, 2, 103, rmq.NewRmqID(104)) + ch <- msg.IntoImmutableMessage(rmq.NewRmqID(108)) + scanner.EXPECT().Chan().RunAndReturn(func() <-chan message.ImmutableMessage { + return ch + }) + scanner.EXPECT().Close().RunAndReturn(func() error { + handler.Close() + return nil + }) + return scanner, nil + }) + if maybe { + read.Maybe() + } + return w +} 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 5f50362b0d2dd..0a59213d6778b 100644 --- a/internal/streamingnode/server/server.go +++ b/internal/streamingnode/server/server.go @@ -6,7 +6,6 @@ import ( "google.golang.org/grpc" "github.com/milvus-io/milvus/internal/streamingnode/client/handler/registry" - "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "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" @@ -45,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. @@ -56,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. diff --git a/internal/streamingnode/server/wal/adaptor/builder.go b/internal/streamingnode/server/wal/adaptor/builder.go index 70e629ce7d6fe..86cc8d4d76477 100644 --- a/internal/streamingnode/server/wal/adaptor/builder.go +++ b/internal/streamingnode/server/wal/adaptor/builder.go @@ -3,23 +3,21 @@ 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/redo" - "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment" - "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick" "github.com/milvus-io/milvus/pkg/streaming/walimpls" ) var _ wal.OpenerBuilder = (*builderAdaptorImpl)(nil) -func AdaptImplsToBuilder(builder walimpls.OpenerBuilderImpls) wal.OpenerBuilder { +func AdaptImplsToBuilder(builder walimpls.OpenerBuilderImpls, interceptorBuilders ...interceptors.InterceptorBuilder) wal.OpenerBuilder { return builderAdaptorImpl{ - builder: builder, + builder: builder, + interceptorBuilders: interceptorBuilders, } } type builderAdaptorImpl struct { - builder walimpls.OpenerBuilderImpls + builder walimpls.OpenerBuilderImpls + interceptorBuilders []interceptors.InterceptorBuilder } func (b builderAdaptorImpl) Name() string { @@ -32,10 +30,5 @@ func (b builderAdaptorImpl) Build() (wal.Opener, error) { return nil, err } // Add all interceptor here. - return adaptImplsToOpener(o, []interceptors.InterceptorBuilder{ - redo.NewInterceptorBuilder(), - timetick.NewInterceptorBuilder(), - segment.NewInterceptorBuilder(), - ddl.NewInterceptorBuilder(), - }), nil + return adaptImplsToOpener(o, b.interceptorBuilders), nil } diff --git a/internal/streamingnode/server/wal/adaptor/scanner_adaptor.go b/internal/streamingnode/server/wal/adaptor/scanner_adaptor.go index 5ee8c6c2f35db..c1459f12159e4 100644 --- a/internal/streamingnode/server/wal/adaptor/scanner_adaptor.go +++ b/internal/streamingnode/server/wal/adaptor/scanner_adaptor.go @@ -29,9 +29,6 @@ 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 = adaptor.ChanMessageHandler(make(chan message.ImmutableMessage)) } @@ -181,7 +178,7 @@ func (s *scannerAdaptorImpl) handleUpstream(msg message.ImmutableMessage) { // 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. diff --git a/internal/streamingnode/server/wal/adaptor/scanner_adaptor_test.go b/internal/streamingnode/server/wal/adaptor/scanner_adaptor_test.go index e80bfffddd1ad..e62561ce97d20 100644 --- a/internal/streamingnode/server/wal/adaptor/scanner_adaptor_test.go +++ b/internal/streamingnode/server/wal/adaptor/scanner_adaptor_test.go @@ -20,16 +20,6 @@ func TestScannerAdaptorReadError(t *testing.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", diff --git a/internal/streamingnode/server/wal/adaptor/wal_adaptor.go b/internal/streamingnode/server/wal/adaptor/wal_adaptor.go index 17e0909b4e7a4..88aef187a2eba 100644 --- a/internal/streamingnode/server/wal/adaptor/wal_adaptor.go +++ b/internal/streamingnode/server/wal/adaptor/wal_adaptor.go @@ -223,7 +223,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..5a15161f0365a 100644 --- a/internal/streamingnode/server/wal/adaptor/wal_adaptor_test.go +++ b/internal/streamingnode/server/wal/adaptor/wal_adaptor_test.go @@ -10,6 +10,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" + "github.com/milvus-io/milvus/internal/mocks/mock_metastore" "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" @@ -45,7 +46,10 @@ func TestWalAdaptorReadFail(t *testing.T) { } func TestWALAdaptor(t *testing.T) { - resource.InitForTest(t) + snMeta := mock_metastore.NewMockStreamingNodeCataLog(t) + snMeta.EXPECT().GetConsumeCheckpoint(mock.Anything, mock.Anything).Return(nil, nil).Maybe() + snMeta.EXPECT().SaveConsumeCheckpoint(mock.Anything, mock.Anything, mock.Anything).Return(nil).Maybe() + resource.InitForTest(t, resource.OptStreamingNodeCatalog(snMeta)) operator := mock_inspector.NewMockTimeTickSyncOperator(t) operator.EXPECT().TimeTickNotifier().Return(inspector.NewTimeTickNotifier()) diff --git a/internal/streamingnode/server/wal/adaptor/wal_test.go b/internal/streamingnode/server/wal/adaptor/wal_test.go index 3b49bf1933882..05188b42351fd 100644 --- a/internal/streamingnode/server/wal/adaptor/wal_test.go +++ b/internal/streamingnode/server/wal/adaptor/wal_test.go @@ -17,9 +17,11 @@ 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/interceptors/redo" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/segment" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors/timetick" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/registry" internaltypes "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/idalloc" @@ -42,7 +44,13 @@ type walTestFramework struct { func TestWAL(t *testing.T) { initResourceForTest(t) - b := registry.MustGetBuilder(walimplstest.WALName) + b := registry.MustGetBuilder(walimplstest.WALName, + redo.NewInterceptorBuilder(), + // TODO: current flusher interceptor cannot work well with the walimplstest. + // flusher.NewInterceptorBuilder(), + timetick.NewInterceptorBuilder(), + segment.NewInterceptorBuilder(), + ) f := &walTestFramework{ b: b, t: t, @@ -68,17 +76,10 @@ func initResourceForTest(t *testing.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), ) } 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/timetick_interceptor.go b/internal/streamingnode/server/wal/interceptors/timetick/timetick_interceptor.go index 15fcf5af44b77..d9d9a3404ada5 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 { diff --git a/internal/streamingnode/server/wal/registry/registry.go b/internal/streamingnode/server/wal/registry/registry.go index 32798228ff962..2f7120706419c 100644 --- a/internal/streamingnode/server/wal/registry/registry.go +++ b/internal/streamingnode/server/wal/registry/registry.go @@ -3,11 +3,12 @@ package registry import ( "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/adaptor" + "github.com/milvus-io/milvus/internal/streamingnode/server/wal/interceptors" "github.com/milvus-io/milvus/pkg/streaming/walimpls/registry" ) // MustGetBuilder returns the wal builder by name. -func MustGetBuilder(name string) wal.OpenerBuilder { +func MustGetBuilder(name string, interceptorBuilders ...interceptors.InterceptorBuilder) wal.OpenerBuilder { b := registry.MustGetBuilder(name) - return adaptor.AdaptImplsToBuilder(b) + return adaptor.AdaptImplsToBuilder(b, interceptorBuilders...) } diff --git a/internal/streamingnode/server/wal/scanner.go b/internal/streamingnode/server/wal/scanner.go index 8396e00fd1d5e..334fe6e9732e4 100644 --- a/internal/streamingnode/server/wal/scanner.go +++ b/internal/streamingnode/server/wal/scanner.go @@ -14,7 +14,9 @@ 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 message.Handler // message handler for message processing. diff --git a/internal/streamingnode/server/walmanager/manager_impl.go b/internal/streamingnode/server/walmanager/manager_impl.go index 84817c9c49134..4c3b177c3e367 100644 --- a/internal/streamingnode/server/walmanager/manager_impl.go +++ b/internal/streamingnode/server/walmanager/manager_impl.go @@ -7,6 +7,10 @@ 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/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" "github.com/milvus-io/milvus/internal/streamingnode/server/wal/registry" "github.com/milvus-io/milvus/internal/util/streamingutil/status" "github.com/milvus-io/milvus/internal/util/streamingutil/util" @@ -21,7 +25,12 @@ var errWALManagerClosed = status.NewOnShutdownError("wal manager is closed") func OpenManager() (Manager, error) { walName := util.MustSelectWALName() resource.Resource().Logger().Info("open wal manager", zap.String("walName", walName)) - opener, err := registry.MustGetBuilder(walName).Build() + opener, err := registry.MustGetBuilder(walName, + redo.NewInterceptorBuilder(), + flusher.NewInterceptorBuilder(), + timetick.NewInterceptorBuilder(), + segment.NewInterceptorBuilder(), + ).Build() if err != nil { return nil, err } 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 3e83ee910f8c9..d0e992f253cc8 100644 --- a/pkg/proto/streaming.proto +++ b/pkg/proto/streaming.proto @@ -549,3 +549,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 f233cdb8a7973..a0dce7460a269 100644 --- a/pkg/proto/streamingpb/streaming.pb.go +++ b/pkg/proto/streamingpb/streaming.pb.go @@ -3883,6 +3883,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[60] + 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[60] + 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{60} +} + +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{ @@ -4411,162 +4459,167 @@ var file_streaming_proto_rawDesc = []byte{ 0x75, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x37, 0x0a, 0x18, 0x63, 0x72, 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, 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, 0x9a, 0x01, 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, - 0x12, 0x21, 0x0a, 0x1d, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54, 0x41, - 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x57, 0x41, 0x49, 0x54, 0x5f, 0x41, 0x43, - 0x4b, 0x10, 0x03, 0x2a, 0x82, 0x04, 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, 0x24, 0x0a, 0x20, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x49, - 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, 0x45, - 0x5f, 0x41, 0x43, 0x51, 0x55, 0x49, 0x52, 0x45, 0x44, 0x10, 0x0c, 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, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x54, 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, 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, 0x9a, 0x01, 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, 0x12, 0x21, 0x0a, 0x1d, 0x42, 0x52, 0x4f, 0x41, 0x44, 0x43, 0x41, 0x53, 0x54, 0x5f, 0x54, + 0x41, 0x53, 0x4b, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x57, 0x41, 0x49, 0x54, 0x5f, 0x41, + 0x43, 0x4b, 0x10, 0x03, 0x2a, 0x82, 0x04, 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, 0x24, 0x0a, 0x20, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, + 0x49, 0x4e, 0x47, 0x5f, 0x43, 0x4f, 0x44, 0x45, 0x5f, 0x52, 0x45, 0x53, 0x4f, 0x55, 0x52, 0x43, + 0x45, 0x5f, 0x41, 0x43, 0x51, 0x55, 0x49, 0x52, 0x45, 0x44, 0x10, 0x0c, 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, 0xd6, 0x02, 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, 0x12, 0x62, 0x0a, 0x03, 0x41, 0x63, 0x6b, 0x12, 0x2b, 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, 0x41, 0x63, 0x6b, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 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, 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, 0xd6, 0x02, + 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, 0x41, 0x63, 0x6b, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6c, 0x0a, 0x05, 0x57, 0x61, 0x74, 0x63, 0x68, - 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x62, 0x0a, 0x03, 0x41, 0x63, 0x6b, 0x12, 0x2b, 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, 0x41, 0x63, + 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 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, 0x41, 0x63, 0x6b, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6c, 0x0a, 0x05, 0x57, 0x61, 0x74, 0x63, + 0x68, 0x12, 0x2d, 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, 0x57, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x2e, 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, 0x57, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x2e, 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, 0x57, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x28, 0x01, 0x30, 0x01, 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, + 0x61, 0x73, 0x74, 0x57, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 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, - 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, 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, 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, + 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, 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, 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, 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, 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, + 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, } var ( @@ -4582,7 +4635,7 @@ func file_streaming_proto_rawDescGZIP() []byte { } var file_streaming_proto_enumTypes = make([]protoimpl.EnumInfo, 4) -var file_streaming_proto_msgTypes = make([]protoimpl.MessageInfo, 61) +var file_streaming_proto_msgTypes = make([]protoimpl.MessageInfo, 62) var file_streaming_proto_goTypes = []interface{}{ (PChannelMetaState)(0), // 0: milvus.proto.streaming.PChannelMetaState (BroadcastTaskState)(0), // 1: milvus.proto.streaming.BroadcastTaskState @@ -4648,17 +4701,18 @@ var file_streaming_proto_goTypes = []interface{}{ (*StreamingNodeManagerCollectStatusResponse)(nil), // 61: milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse (*SegmentAssignmentMeta)(nil), // 62: milvus.proto.streaming.SegmentAssignmentMeta (*SegmentAssignmentStat)(nil), // 63: milvus.proto.streaming.SegmentAssignmentStat - nil, // 64: milvus.proto.streaming.BroadcastResponse.ResultsEntry - (*messagespb.Message)(nil), // 65: milvus.proto.messages.Message - (*messagespb.BroadcastEvent)(nil), // 66: milvus.proto.messages.BroadcastEvent - (*emptypb.Empty)(nil), // 67: google.protobuf.Empty - (*messagespb.MessageID)(nil), // 68: milvus.proto.messages.MessageID - (messagespb.MessageType)(0), // 69: milvus.proto.messages.MessageType - (*messagespb.TxnContext)(nil), // 70: milvus.proto.messages.TxnContext - (*anypb.Any)(nil), // 71: google.protobuf.Any - (*messagespb.ImmutableMessage)(nil), // 72: milvus.proto.messages.ImmutableMessage - (*milvuspb.GetComponentStatesRequest)(nil), // 73: milvus.proto.milvus.GetComponentStatesRequest - (*milvuspb.ComponentStates)(nil), // 74: milvus.proto.milvus.ComponentStates + (*WALCheckpoint)(nil), // 64: milvus.proto.streaming.WALCheckpoint + nil, // 65: milvus.proto.streaming.BroadcastResponse.ResultsEntry + (*messagespb.Message)(nil), // 66: milvus.proto.messages.Message + (*messagespb.BroadcastEvent)(nil), // 67: milvus.proto.messages.BroadcastEvent + (*emptypb.Empty)(nil), // 68: google.protobuf.Empty + (*messagespb.MessageID)(nil), // 69: milvus.proto.messages.MessageID + (messagespb.MessageType)(0), // 70: milvus.proto.messages.MessageType + (*messagespb.TxnContext)(nil), // 71: milvus.proto.messages.TxnContext + (*anypb.Any)(nil), // 72: google.protobuf.Any + (*messagespb.ImmutableMessage)(nil), // 73: milvus.proto.messages.ImmutableMessage + (*milvuspb.GetComponentStatesRequest)(nil), // 74: milvus.proto.milvus.GetComponentStatesRequest + (*milvuspb.ComponentStates)(nil), // 75: milvus.proto.milvus.ComponentStates } var file_streaming_proto_depIdxs = []int32{ 25, // 0: milvus.proto.streaming.PChannelAssignmentLog.node:type_name -> milvus.proto.streaming.StreamingNodeInfo @@ -4666,16 +4720,16 @@ var file_streaming_proto_depIdxs = []int32{ 25, // 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 - 65, // 5: milvus.proto.streaming.BroadcastTask.message:type_name -> milvus.proto.messages.Message + 66, // 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 - 65, // 7: milvus.proto.streaming.BroadcastRequest.message:type_name -> milvus.proto.messages.Message - 64, // 8: milvus.proto.streaming.BroadcastResponse.results:type_name -> milvus.proto.streaming.BroadcastResponse.ResultsEntry + 66, // 7: milvus.proto.streaming.BroadcastRequest.message:type_name -> milvus.proto.messages.Message + 65, // 8: milvus.proto.streaming.BroadcastResponse.results:type_name -> milvus.proto.streaming.BroadcastResponse.ResultsEntry 14, // 9: milvus.proto.streaming.BroadcastWatchRequest.create_event_watch:type_name -> milvus.proto.streaming.BroadcastCreateEventWatchRequest 15, // 10: milvus.proto.streaming.BroadcastWatchRequest.close:type_name -> milvus.proto.streaming.CloseBroadcastWatchRequest - 66, // 11: milvus.proto.streaming.BroadcastCreateEventWatchRequest.event:type_name -> milvus.proto.messages.BroadcastEvent + 67, // 11: milvus.proto.streaming.BroadcastCreateEventWatchRequest.event:type_name -> milvus.proto.messages.BroadcastEvent 17, // 12: milvus.proto.streaming.BroadcastWatchResponse.event_done:type_name -> milvus.proto.streaming.BroadcastEventWatchResponse 18, // 13: milvus.proto.streaming.BroadcastWatchResponse.close:type_name -> milvus.proto.streaming.CloseBroadcastWatchResponse - 66, // 14: milvus.proto.streaming.BroadcastEventWatchResponse.event:type_name -> milvus.proto.messages.BroadcastEvent + 67, // 14: milvus.proto.streaming.BroadcastEventWatchResponse.event:type_name -> milvus.proto.messages.BroadcastEvent 20, // 15: milvus.proto.streaming.AssignmentDiscoverRequest.report_error:type_name -> milvus.proto.streaming.ReportAssignmentErrorRequest 21, // 16: milvus.proto.streaming.AssignmentDiscoverRequest.close:type_name -> milvus.proto.streaming.CloseAssignmentDiscoverRequest 4, // 17: milvus.proto.streaming.ReportAssignmentErrorRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo @@ -4686,27 +4740,27 @@ var file_streaming_proto_depIdxs = []int32{ 26, // 22: milvus.proto.streaming.FullStreamingNodeAssignmentWithVersion.assignments:type_name -> milvus.proto.streaming.StreamingNodeAssignment 25, // 23: milvus.proto.streaming.StreamingNodeAssignment.node:type_name -> milvus.proto.streaming.StreamingNodeInfo 4, // 24: milvus.proto.streaming.StreamingNodeAssignment.channels:type_name -> milvus.proto.streaming.PChannelInfo - 67, // 25: milvus.proto.streaming.DeliverPolicy.all:type_name -> google.protobuf.Empty - 67, // 26: milvus.proto.streaming.DeliverPolicy.latest:type_name -> google.protobuf.Empty - 68, // 27: milvus.proto.streaming.DeliverPolicy.start_from:type_name -> milvus.proto.messages.MessageID - 68, // 28: milvus.proto.streaming.DeliverPolicy.start_after:type_name -> milvus.proto.messages.MessageID + 68, // 25: milvus.proto.streaming.DeliverPolicy.all:type_name -> google.protobuf.Empty + 68, // 26: milvus.proto.streaming.DeliverPolicy.latest:type_name -> google.protobuf.Empty + 69, // 27: milvus.proto.streaming.DeliverPolicy.start_from:type_name -> milvus.proto.messages.MessageID + 69, // 28: milvus.proto.streaming.DeliverPolicy.start_after:type_name -> milvus.proto.messages.MessageID 29, // 29: milvus.proto.streaming.DeliverFilter.time_tick_gt:type_name -> milvus.proto.streaming.DeliverFilterTimeTickGT 30, // 30: milvus.proto.streaming.DeliverFilter.time_tick_gte:type_name -> milvus.proto.streaming.DeliverFilterTimeTickGTE 31, // 31: milvus.proto.streaming.DeliverFilter.message_type:type_name -> milvus.proto.streaming.DeliverFilterMessageType - 69, // 32: milvus.proto.streaming.DeliverFilterMessageType.message_types:type_name -> milvus.proto.messages.MessageType + 70, // 32: milvus.proto.streaming.DeliverFilterMessageType.message_types:type_name -> milvus.proto.messages.MessageType 2, // 33: milvus.proto.streaming.StreamingError.code:type_name -> milvus.proto.streaming.StreamingCode 35, // 34: milvus.proto.streaming.ProduceRequest.produce:type_name -> milvus.proto.streaming.ProduceMessageRequest 36, // 35: milvus.proto.streaming.ProduceRequest.close:type_name -> milvus.proto.streaming.CloseProducerRequest 4, // 36: milvus.proto.streaming.CreateProducerRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo - 65, // 37: milvus.proto.streaming.ProduceMessageRequest.message:type_name -> milvus.proto.messages.Message + 66, // 37: milvus.proto.streaming.ProduceMessageRequest.message:type_name -> milvus.proto.messages.Message 38, // 38: milvus.proto.streaming.ProduceResponse.create:type_name -> milvus.proto.streaming.CreateProducerResponse 39, // 39: milvus.proto.streaming.ProduceResponse.produce:type_name -> milvus.proto.streaming.ProduceMessageResponse 41, // 40: milvus.proto.streaming.ProduceResponse.close:type_name -> milvus.proto.streaming.CloseProducerResponse 40, // 41: milvus.proto.streaming.ProduceMessageResponse.result:type_name -> milvus.proto.streaming.ProduceMessageResponseResult 32, // 42: milvus.proto.streaming.ProduceMessageResponse.error:type_name -> milvus.proto.streaming.StreamingError - 68, // 43: milvus.proto.streaming.ProduceMessageResponseResult.id:type_name -> milvus.proto.messages.MessageID - 70, // 44: milvus.proto.streaming.ProduceMessageResponseResult.txnContext:type_name -> milvus.proto.messages.TxnContext - 71, // 45: milvus.proto.streaming.ProduceMessageResponseResult.extra:type_name -> google.protobuf.Any + 69, // 43: milvus.proto.streaming.ProduceMessageResponseResult.id:type_name -> milvus.proto.messages.MessageID + 71, // 44: milvus.proto.streaming.ProduceMessageResponseResult.txnContext:type_name -> milvus.proto.messages.TxnContext + 72, // 45: milvus.proto.streaming.ProduceMessageResponseResult.extra:type_name -> google.protobuf.Any 46, // 46: milvus.proto.streaming.ConsumeRequest.create_vchannel_consumer:type_name -> milvus.proto.streaming.CreateVChannelConsumerRequest 45, // 47: milvus.proto.streaming.ConsumeRequest.create_vchannel_consumers:type_name -> milvus.proto.streaming.CreateVChannelConsumersRequest 49, // 48: milvus.proto.streaming.ConsumeRequest.close_vchannel:type_name -> milvus.proto.streaming.CloseVChannelConsumerRequest @@ -4723,38 +4777,39 @@ var file_streaming_proto_depIdxs = []int32{ 47, // 59: milvus.proto.streaming.ConsumeResponse.create_vchannels:type_name -> milvus.proto.streaming.CreateVChannelConsumersResponse 50, // 60: milvus.proto.streaming.ConsumeResponse.close_vchannel:type_name -> milvus.proto.streaming.CloseVChannelConsumerResponse 54, // 61: milvus.proto.streaming.ConsumeResponse.close:type_name -> milvus.proto.streaming.CloseConsumerResponse - 72, // 62: milvus.proto.streaming.ConsumeMessageReponse.message:type_name -> milvus.proto.messages.ImmutableMessage + 73, // 62: milvus.proto.streaming.ConsumeMessageReponse.message:type_name -> milvus.proto.messages.ImmutableMessage 4, // 63: milvus.proto.streaming.StreamingNodeManagerAssignRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo 4, // 64: milvus.proto.streaming.StreamingNodeManagerRemoveRequest.pchannel:type_name -> milvus.proto.streaming.PChannelInfo 60, // 65: milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse.balance_attributes:type_name -> milvus.proto.streaming.StreamingNodeBalanceAttributes 3, // 66: milvus.proto.streaming.SegmentAssignmentMeta.state:type_name -> milvus.proto.streaming.SegmentAssignmentState 63, // 67: milvus.proto.streaming.SegmentAssignmentMeta.stat:type_name -> milvus.proto.streaming.SegmentAssignmentStat - 40, // 68: milvus.proto.streaming.BroadcastResponse.ResultsEntry.value:type_name -> milvus.proto.streaming.ProduceMessageResponseResult - 73, // 69: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest - 9, // 70: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:input_type -> milvus.proto.streaming.BroadcastRequest - 11, // 71: milvus.proto.streaming.StreamingCoordBroadcastService.Ack:input_type -> milvus.proto.streaming.BroadcastAckRequest - 13, // 72: milvus.proto.streaming.StreamingCoordBroadcastService.Watch:input_type -> milvus.proto.streaming.BroadcastWatchRequest - 19, // 73: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:input_type -> milvus.proto.streaming.AssignmentDiscoverRequest - 33, // 74: milvus.proto.streaming.StreamingNodeHandlerService.Produce:input_type -> milvus.proto.streaming.ProduceRequest - 42, // 75: milvus.proto.streaming.StreamingNodeHandlerService.Consume:input_type -> milvus.proto.streaming.ConsumeRequest - 55, // 76: milvus.proto.streaming.StreamingNodeManagerService.Assign:input_type -> milvus.proto.streaming.StreamingNodeManagerAssignRequest - 57, // 77: milvus.proto.streaming.StreamingNodeManagerService.Remove:input_type -> milvus.proto.streaming.StreamingNodeManagerRemoveRequest - 59, // 78: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:input_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusRequest - 74, // 79: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates - 10, // 80: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:output_type -> milvus.proto.streaming.BroadcastResponse - 12, // 81: milvus.proto.streaming.StreamingCoordBroadcastService.Ack:output_type -> milvus.proto.streaming.BroadcastAckResponse - 16, // 82: milvus.proto.streaming.StreamingCoordBroadcastService.Watch:output_type -> milvus.proto.streaming.BroadcastWatchResponse - 22, // 83: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:output_type -> milvus.proto.streaming.AssignmentDiscoverResponse - 37, // 84: milvus.proto.streaming.StreamingNodeHandlerService.Produce:output_type -> milvus.proto.streaming.ProduceResponse - 51, // 85: milvus.proto.streaming.StreamingNodeHandlerService.Consume:output_type -> milvus.proto.streaming.ConsumeResponse - 56, // 86: milvus.proto.streaming.StreamingNodeManagerService.Assign:output_type -> milvus.proto.streaming.StreamingNodeManagerAssignResponse - 58, // 87: milvus.proto.streaming.StreamingNodeManagerService.Remove:output_type -> milvus.proto.streaming.StreamingNodeManagerRemoveResponse - 61, // 88: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:output_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse - 79, // [79:89] is the sub-list for method output_type - 69, // [69:79] is the sub-list for method input_type - 69, // [69:69] is the sub-list for extension type_name - 69, // [69:69] is the sub-list for extension extendee - 0, // [0:69] is the sub-list for field type_name + 69, // 68: milvus.proto.streaming.WALCheckpoint.messageID:type_name -> milvus.proto.messages.MessageID + 40, // 69: milvus.proto.streaming.BroadcastResponse.ResultsEntry.value:type_name -> milvus.proto.streaming.ProduceMessageResponseResult + 74, // 70: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 9, // 71: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:input_type -> milvus.proto.streaming.BroadcastRequest + 11, // 72: milvus.proto.streaming.StreamingCoordBroadcastService.Ack:input_type -> milvus.proto.streaming.BroadcastAckRequest + 13, // 73: milvus.proto.streaming.StreamingCoordBroadcastService.Watch:input_type -> milvus.proto.streaming.BroadcastWatchRequest + 19, // 74: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:input_type -> milvus.proto.streaming.AssignmentDiscoverRequest + 33, // 75: milvus.proto.streaming.StreamingNodeHandlerService.Produce:input_type -> milvus.proto.streaming.ProduceRequest + 42, // 76: milvus.proto.streaming.StreamingNodeHandlerService.Consume:input_type -> milvus.proto.streaming.ConsumeRequest + 55, // 77: milvus.proto.streaming.StreamingNodeManagerService.Assign:input_type -> milvus.proto.streaming.StreamingNodeManagerAssignRequest + 57, // 78: milvus.proto.streaming.StreamingNodeManagerService.Remove:input_type -> milvus.proto.streaming.StreamingNodeManagerRemoveRequest + 59, // 79: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:input_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusRequest + 75, // 80: milvus.proto.streaming.StreamingNodeStateService.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 10, // 81: milvus.proto.streaming.StreamingCoordBroadcastService.Broadcast:output_type -> milvus.proto.streaming.BroadcastResponse + 12, // 82: milvus.proto.streaming.StreamingCoordBroadcastService.Ack:output_type -> milvus.proto.streaming.BroadcastAckResponse + 16, // 83: milvus.proto.streaming.StreamingCoordBroadcastService.Watch:output_type -> milvus.proto.streaming.BroadcastWatchResponse + 22, // 84: milvus.proto.streaming.StreamingCoordAssignmentService.AssignmentDiscover:output_type -> milvus.proto.streaming.AssignmentDiscoverResponse + 37, // 85: milvus.proto.streaming.StreamingNodeHandlerService.Produce:output_type -> milvus.proto.streaming.ProduceResponse + 51, // 86: milvus.proto.streaming.StreamingNodeHandlerService.Consume:output_type -> milvus.proto.streaming.ConsumeResponse + 56, // 87: milvus.proto.streaming.StreamingNodeManagerService.Assign:output_type -> milvus.proto.streaming.StreamingNodeManagerAssignResponse + 58, // 88: milvus.proto.streaming.StreamingNodeManagerService.Remove:output_type -> milvus.proto.streaming.StreamingNodeManagerRemoveResponse + 61, // 89: milvus.proto.streaming.StreamingNodeManagerService.CollectStatus:output_type -> milvus.proto.streaming.StreamingNodeManagerCollectStatusResponse + 80, // [80:90] is the sub-list for method output_type + 70, // [70:80] is the sub-list for method input_type + 70, // [70:70] is the sub-list for extension type_name + 70, // [70:70] is the sub-list for extension extendee + 0, // [0:70] is the sub-list for field type_name } func init() { file_streaming_proto_init() } @@ -5483,6 +5538,18 @@ func file_streaming_proto_init() { return nil } } + file_streaming_proto_msgTypes[60].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[9].OneofWrappers = []interface{}{ (*BroadcastWatchRequest_CreateEventWatch)(nil), @@ -5548,7 +5615,7 @@ func file_streaming_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_streaming_proto_rawDesc, NumEnums: 4, - NumMessages: 61, + NumMessages: 62, NumExtensions: 0, NumServices: 5, }, diff --git a/pkg/streaming/util/message/adaptor/handler.go b/pkg/streaming/util/message/adaptor/handler.go index 2d9ae00c64910..6efc9495b3cef 100644 --- a/pkg/streaming/util/message/adaptor/handler.go +++ b/pkg/streaming/util/message/adaptor/handler.go @@ -38,17 +38,19 @@ func (d ChanMessageHandler) Close() { // NewMsgPackAdaptorHandler create a new message pack adaptor handler. func NewMsgPackAdaptorHandler() *MsgPackAdaptorHandler { return &MsgPackAdaptorHandler{ - base: NewBaseMsgPackAdaptorHandler(), + channel: make(chan *msgstream.MsgPack), + base: NewBaseMsgPackAdaptorHandler(), } } 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. @@ -63,7 +65,7 @@ func (m *MsgPackAdaptorHandler) Handle(param message.HandleParam) message.Handle for { var sendCh chan<- *msgstream.MsgPack if m.base.PendingMsgPack.Len() != 0 { - sendCh = m.base.Channel + sendCh = m.channel } select { @@ -100,14 +102,13 @@ func (m *MsgPackAdaptorHandler) Handle(param message.HandleParam) message.Handle // 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](), } @@ -116,7 +117,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/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/test_case.go b/pkg/streaming/util/message/test_case.go index 488b01e7c48d7..84999463c649b 100644 --- a/pkg/streaming/util/message/test_case.go +++ b/pkg/streaming/util/message/test_case.go @@ -7,10 +7,12 @@ import ( "testing" "github.com/stretchr/testify/assert" + "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/pkg/proto/messagespb" ) func CreateTestInsertMessage(t *testing.T, segmentID int64, totalRows int, timetick uint64, messageID MessageID) MutableMessage { @@ -102,11 +104,47 @@ func CreateTestInsertMessage(t *testing.T, segmentID int64, totalRows int, timet return msg } +func CreateTestDropCollectionMessage(t *testing.T, collectionID int64, timetick uint64, messageID MessageID) MutableMessage { + header := &DropCollectionMessageHeader{ + CollectionId: collectionID, + } + payload := &msgpb.DropCollectionRequest{ + Base: &commonpb.MsgBase{ + MsgType: commonpb.MsgType_DropCollection, + MsgID: collectionID, + Timestamp: timetick, + }, + DbName: "db", + CollectionName: "collection", + DbID: 1, + CollectionID: collectionID, + } + msg, err := NewDropCollectionMessageBuilderV1(). + WithHeader(header). + WithBody(payload). + WithVChannel("v1"). + BuildMutable() + assert.NoError(t, err) + msg.WithTimeTick(timetick) + msg.WithLastConfirmed(messageID) + return msg +} + func CreateTestCreateCollectionMessage(t *testing.T, collectionID int64, timetick uint64, messageID MessageID) MutableMessage { header := &CreateCollectionMessageHeader{ CollectionId: collectionID, PartitionIds: []int64{2}, } + schema := &schemapb.CollectionSchema{ + Fields: []*schemapb.FieldSchema{ + {FieldID: 100, Name: "ID", IsPrimaryKey: true, DataType: schemapb.DataType_Int64}, + {FieldID: 101, Name: "Vector", DataType: schemapb.DataType_FloatVector}, + }, + } + schemaBytes, err := proto.Marshal(schema) + if err != nil { + panic(err) + } payload := &msgpb.CreateCollectionRequest{ Base: &commonpb.MsgBase{ MsgType: commonpb.MsgType_CreateCollection, @@ -118,6 +156,7 @@ func CreateTestCreateCollectionMessage(t *testing.T, collectionID int64, timetic PartitionName: "partition", DbID: 1, CollectionID: collectionID, + Schema: schemaBytes, } msg, err := NewCreateCollectionMessageBuilderV1(). @@ -131,6 +170,44 @@ func CreateTestCreateCollectionMessage(t *testing.T, collectionID int64, timetic return msg } +func CreateTestCreateSegmentMessage(t *testing.T, collectionID int64, timetick uint64, messageID MessageID) MutableMessage { + payload := &CreateSegmentMessageBody{ + CollectionId: collectionID, + Segments: []*messagespb.CreateSegmentInfo{ + { + PartitionId: 1, + SegmentId: 1, + }, + }, + } + msg, err := NewCreateSegmentMessageBuilderV2(). + WithHeader(&CreateSegmentMessageHeader{}). + WithBody(payload). + WithVChannel("v1"). + BuildMutable() + assert.NoError(t, err) + msg.WithTimeTick(timetick) + msg.WithLastConfirmed(messageID) + return msg +} + +func CreateTestTimeTickSyncMessage(t *testing.T, collectionID int64, timetick uint64, messageID MessageID) MutableMessage { + msg, err := NewTimeTickMessageBuilderV1(). + WithHeader(&TimeTickMessageHeader{}). + WithBody(&msgpb.TimeTickMsg{ + Base: &commonpb.MsgBase{ + MsgType: commonpb.MsgType_TimeTick, + Timestamp: timetick, + }, + }). + WithAllVChannel(). + BuildMutable() + assert.NoError(t, err) + msg.WithTimeTick(timetick) + msg.WithLastConfirmed(messageID) + return msg +} + // CreateTestEmptyInsertMesage creates an empty insert message for testing func CreateTestEmptyInsertMesage(msgID int64, extraProperties map[string]string) MutableMessage { msg, err := NewInsertMessageBuilderV1(). diff --git a/tests/integration/minicluster_v2.go b/tests/integration/minicluster_v2.go index 236b59199e893..222ace49f456d 100644 --- a/tests/integration/minicluster_v2.go +++ b/tests/integration/minicluster_v2.go @@ -79,13 +79,14 @@ func DefaultParams() map[string]string { // Notice: don't use ParamItem.Key here, the config key will be empty before param table init configMap = map[string]string{ - "mq.type": "rocksmq", - "etcd.rootPath": testPath, - "minio.rootPath": testPath, - "localStorage.path": path.Join("/tmp", testPath), - "common.storageType": "local", - "dataNode.memory.forceSyncEnable": "false", // local execution will print too many logs - "common.gracefulStopTimeout": "30", + "mq.type": "rocksmq", + "etcd.rootPath": testPath, + "msgChannel.chanNamePrefix.cluster": testPath, + "minio.rootPath": testPath, + "localStorage.path": path.Join("/tmp", testPath), + "common.storageType": "local", + "dataNode.memory.forceSyncEnable": "false", // local execution will print too many logs + "common.gracefulStopTimeout": "30", } })