enhance: make pchannel level flusher (#39275)

issue: #38399

- Add a pchannel level checkpoint for flush processing
- Refactor the recovery of flushers of wal
- make a shared wal scanner first, then make multi datasyncservice on it

Signed-off-by: chyezh <chyezh@outlook.com>
pull/39754/head
Zhen Ye 2025-02-10 16:32:45 +08:00 committed by GitHub
parent 7f119a7997
commit d3e32bb599
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
54 changed files with 2727 additions and 1349 deletions

View File

@ -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:

View File

@ -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
}

View File

@ -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)
}

View File

@ -50,6 +50,7 @@ type ChannelCheckpointUpdater struct {
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)
}

View File

@ -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
}

View File

@ -5,4 +5,6 @@ const (
DirectoryWAL = "wal"
DirectorySegmentAssign = "segment-assign"
KeyConsumeCheckpoint = "consume-checkpoint"
)

View File

@ -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) + "/"

View File

@ -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)

View File

@ -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
}

View File

@ -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
}

View File

@ -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 {

View File

@ -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))

View File

@ -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 {

View File

@ -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,

View File

@ -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()
}

View File

@ -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))
}
}

View File

@ -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()
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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())
}

View File

@ -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()
}

View File

@ -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()
}

View File

@ -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
}

View File

@ -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)
}

View File

@ -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)
}

View File

@ -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)
}

View File

@ -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
}

View File

@ -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

View File

@ -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]()),

View File

@ -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.

View File

@ -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,
interceptorBuilders: interceptorBuilders,
}
}
type builderAdaptorImpl struct {
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
}

View File

@ -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.

View File

@ -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",

View File

@ -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()

View File

@ -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())

View File

@ -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),
)
}

View File

@ -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
}

View File

@ -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() {}

View File

@ -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,
}
}

View File

@ -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() {}

View File

@ -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 {

View File

@ -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 {

View File

@ -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...)
}

View File

@ -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.

View File

@ -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
}

View File

@ -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),
)

View File

@ -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))
}

View File

@ -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),
)

View File

@ -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;
}

View File

@ -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,
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,
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,
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, 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,
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, 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,
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, 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, 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,
},

View File

@ -38,17 +38,19 @@ func (d ChanMessageHandler) Close() {
// NewMsgPackAdaptorHandler create a new message pack adaptor handler.
func NewMsgPackAdaptorHandler() *MsgPackAdaptorHandler {
return &MsgPackAdaptorHandler{
channel: make(chan *msgstream.MsgPack),
base: NewBaseMsgPackAdaptorHandler(),
}
}
type MsgPackAdaptorHandler struct {
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.
}

View File

@ -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)

View File

@ -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().

View File

@ -81,6 +81,7 @@ func DefaultParams() map[string]string {
configMap = map[string]string{
"mq.type": "rocksmq",
"etcd.rootPath": testPath,
"msgChannel.chanNamePrefix.cluster": testPath,
"minio.rootPath": testPath,
"localStorage.path": path.Join("/tmp", testPath),
"common.storageType": "local",