mirror of https://github.com/milvus-io/milvus.git
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
parent
7f119a7997
commit
d3e32bb599
|
@ -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:
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -5,4 +5,6 @@ const (
|
|||
|
||||
DirectoryWAL = "wal"
|
||||
DirectorySegmentAssign = "segment-assign"
|
||||
|
||||
KeyConsumeCheckpoint = "consume-checkpoint"
|
||||
)
|
||||
|
|
|
@ -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) + "/"
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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 {
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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()
|
||||
}
|
|
@ -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))
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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())
|
||||
}
|
|
@ -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()
|
||||
}
|
|
@ -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()
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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]()),
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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()
|
||||
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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),
|
||||
)
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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() {}
|
|
@ -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,
|
||||
}
|
||||
}
|
|
@ -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() {}
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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...)
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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),
|
||||
)
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
|
|
|
@ -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),
|
||||
)
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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,
|
||||
},
|
||||
|
|
|
@ -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.
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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().
|
||||
|
|
|
@ -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",
|
||||
|
|
Loading…
Reference in New Issue