enhance: add schema version at recovery storage (#43500)

issue: #43072, #43289

- manage the schema version at recovery storage.
- update the schema when creating collection or alter schema.
- get schema at write buffer based on version.
- recover the schema when upgrading from 2.5.

---------

Signed-off-by: chyezh <chyezh@outlook.com>
This commit is contained in:
Zhen Ye 2025-07-23 21:38:54 +08:00 committed by GitHub
parent 9fbd41a97d
commit e9ab73e93d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
59 changed files with 3416 additions and 3746 deletions

View File

@ -31,7 +31,7 @@ import (
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
func LoadBM25Stats(ctx context.Context, chunkManager storage.ChunkManager, schema *schemapb.CollectionSchema, segmentID int64, statsBinlogs []*datapb.FieldBinlog) (map[int64]*storage.BM25Stats, error) {
func LoadBM25Stats(ctx context.Context, chunkManager storage.ChunkManager, segmentID int64, statsBinlogs []*datapb.FieldBinlog) (map[int64]*storage.BM25Stats, error) {
startTs := time.Now()
log := log.With(zap.Int64("segmentID", segmentID))
log.Info("begin to reload history BM25 stats", zap.Int("statsBinLogsLen", len(statsBinlogs)))

View File

@ -1012,15 +1012,8 @@ func (s *Server) GetChannelRecoveryInfo(ctx context.Context, req *datapb.GetChan
return resp, nil
}
collectionID := funcutil.GetCollectionIDFromVChannel(req.GetVchannel())
// `handler.GetCollection` cannot fetch dropping collection,
// so we use `broker.DescribeCollectionInternal` to get collection info to help fetch dropping collection to get the recovery info.
collection, err := s.broker.DescribeCollectionInternal(ctx, collectionID)
if err := merr.CheckRPCCall(collection, err); err != nil {
resp.Status = merr.Status(err)
return resp, nil
}
channel := NewRWChannel(req.GetVchannel(), collectionID, nil, collection.Schema, 0, nil) // TODO: remove RWChannel, just use vchannel + collectionID
channel := NewRWChannel(req.GetVchannel(), collectionID, nil, nil, 0, nil) // TODO: remove RWChannel, just use vchannel + collectionID
channelInfo := s.handler.GetDataVChanPositions(channel, allPartitionID)
if channelInfo.SeekPosition == nil {
log.Warn("channel recovery start position is not found, may collection is on creating")
@ -1050,7 +1043,7 @@ func (s *Server) GetChannelRecoveryInfo(ctx context.Context, req *datapb.GetChan
)
resp.Info = channelInfo
resp.Schema = collection.Schema
resp.Schema = nil // schema is managed by streaming node itself now.
resp.SegmentsNotCreatedByStreaming = segmentsNotCreatedByStreaming
return resp, nil
}

View File

@ -1522,13 +1522,7 @@ func TestGetChannelRecoveryInfo(t *testing.T) {
// get collection failed
broker := broker.NewMockBroker(t)
broker.EXPECT().DescribeCollectionInternal(mock.Anything, mock.Anything).Return(nil, errors.New("mock err"))
s.broker = broker
resp, err = s.GetChannelRecoveryInfo(ctx, &datapb.GetChannelRecoveryInfoRequest{
Vchannel: "ch-1",
})
assert.NoError(t, err)
assert.Error(t, merr.Error(resp.GetStatus()))
// normal case
channelInfo := &datapb.VchannelInfo{
@ -1541,12 +1535,6 @@ func TestGetChannelRecoveryInfo(t *testing.T) {
IndexedSegmentIds: []int64{4},
}
broker.EXPECT().DescribeCollectionInternal(mock.Anything, mock.Anything).Unset()
broker.EXPECT().DescribeCollectionInternal(mock.Anything, mock.Anything).
Return(&milvuspb.DescribeCollectionResponse{
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
Schema: &schemapb.CollectionSchema{},
}, nil)
handler := NewNMockHandler(t)
handler.EXPECT().GetDataVChanPositions(mock.Anything, mock.Anything).Return(channelInfo)
s.handler = handler
@ -1567,7 +1555,7 @@ func TestGetChannelRecoveryInfo(t *testing.T) {
})
assert.NoError(t, err)
assert.Equal(t, int32(0), resp.GetStatus().GetCode())
assert.NotNil(t, resp.GetSchema())
assert.Nil(t, resp.GetSchema())
assert.Equal(t, channelInfo, resp.GetInfo())
}

View File

@ -31,9 +31,7 @@ type PriorityQueue interface {
TaskIDs() []int64
}
var (
_ PriorityQueue = &priorityQueuePolicy{}
)
var _ PriorityQueue = &priorityQueuePolicy{}
// priorityQueuePolicy implements a priority queue that sorts tasks by taskID (smaller taskID has higher priority)
type priorityQueuePolicy struct {

View File

@ -1,534 +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 channel
import (
"context"
"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/util"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/util/lifetime"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
type (
releaseFunc func(channel string)
watchFunc func(ctx context.Context, pipelineParams *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler) (*pipeline.DataSyncService, error)
)
type ChannelManager interface {
Submit(info *datapb.ChannelWatchInfo) error
GetProgress(info *datapb.ChannelWatchInfo) *datapb.ChannelOperationProgressResponse
Close()
Start()
}
type ChannelManagerImpl struct {
mu sync.RWMutex
pipelineParams *util.PipelineParams
fgManager pipeline.FlowgraphManager
communicateCh chan *opState
opRunners *typeutil.ConcurrentMap[string, *opRunner] // channel -> runner
abnormals *typeutil.ConcurrentMap[int64, string] // OpID -> Channel
releaseFunc releaseFunc
closeCh lifetime.SafeChan
closeWaiter sync.WaitGroup
}
func NewChannelManager(pipelineParams *util.PipelineParams, fgManager pipeline.FlowgraphManager) *ChannelManagerImpl {
cm := ChannelManagerImpl{
pipelineParams: pipelineParams,
fgManager: fgManager,
communicateCh: make(chan *opState, 100),
opRunners: typeutil.NewConcurrentMap[string, *opRunner](),
abnormals: typeutil.NewConcurrentMap[int64, string](),
releaseFunc: func(channelName string) {
if pipelineParams.CompactionExecutor != nil {
pipelineParams.CompactionExecutor.DiscardPlan(channelName)
}
// RemoveFlowgraph before remove WriteBuffer, otherwise panics will happen.
fgManager.RemoveFlowgraph(channelName)
if pipelineParams.WriteBufferManager != nil {
pipelineParams.WriteBufferManager.RemoveChannel(channelName)
}
},
closeCh: lifetime.NewSafeChan(),
}
return &cm
}
func (m *ChannelManagerImpl) Submit(info *datapb.ChannelWatchInfo) error {
channel := info.GetVchan().GetChannelName()
// skip enqueue the same operation resubmmited by datacoord
if runner, ok := m.opRunners.Get(channel); ok {
if _, exists := runner.Exist(info.GetOpID()); exists {
log.Warn("op already exist, skip", zap.Int64("opID", info.GetOpID()), zap.String("channel", channel))
return nil
}
}
if info.GetState() == datapb.ChannelWatchState_ToWatch &&
m.fgManager.HasFlowgraphWithOpID(channel, info.GetOpID()) {
log.Warn("Watch op already finished, skip", zap.Int64("opID", info.GetOpID()), zap.String("channel", channel))
return nil
}
// DataNode already watched this channel of other OpID
if info.GetState() == datapb.ChannelWatchState_ToWatch &&
m.fgManager.HasFlowgraph(channel) {
return merr.WrapErrChannelReduplicate(channel)
}
if info.GetState() == datapb.ChannelWatchState_ToRelease &&
!m.fgManager.HasFlowgraph(channel) {
log.Warn("Release op already finished, skip", zap.Int64("opID", info.GetOpID()), zap.String("channel", channel))
return nil
}
runner := m.getOrCreateRunner(channel)
return runner.Enqueue(info)
}
func (m *ChannelManagerImpl) GetProgress(info *datapb.ChannelWatchInfo) *datapb.ChannelOperationProgressResponse {
m.mu.RLock()
defer m.mu.RUnlock()
resp := &datapb.ChannelOperationProgressResponse{
Status: merr.Success(),
OpID: info.GetOpID(),
}
channel := info.GetVchan().GetChannelName()
switch info.GetState() {
case datapb.ChannelWatchState_ToWatch:
// running flowgraph means watch success
if m.fgManager.HasFlowgraphWithOpID(channel, info.GetOpID()) {
resp.State = datapb.ChannelWatchState_WatchSuccess
resp.Progress = 100
return resp
}
if runner, ok := m.opRunners.Get(channel); ok {
if progress, exists := runner.Exist(info.GetOpID()); exists {
resp.Progress = progress
resp.State = datapb.ChannelWatchState_ToWatch
} else {
resp.State = datapb.ChannelWatchState_WatchFailure
}
return resp
}
resp.State = datapb.ChannelWatchState_WatchFailure
return resp
case datapb.ChannelWatchState_ToRelease:
if !m.fgManager.HasFlowgraph(channel) {
resp.State = datapb.ChannelWatchState_ReleaseSuccess
return resp
}
runner, ok := m.opRunners.Get(channel)
if ok {
_, exists := runner.Exist(info.GetOpID())
if exists {
resp.State = datapb.ChannelWatchState_ToRelease
return resp
}
}
resp.State = datapb.ChannelWatchState_ReleaseFailure
return resp
default:
err := merr.WrapErrParameterInvalid("ToWatch or ToRelease", info.GetState().String())
log.Warn("fail to get progress", zap.Error(err))
resp.Status = merr.Status(err)
return resp
}
}
func (m *ChannelManagerImpl) Close() {
if m.opRunners != nil {
m.opRunners.Range(func(channel string, runner *opRunner) bool {
runner.Close()
return true
})
}
m.closeCh.Close()
m.closeWaiter.Wait()
}
func (m *ChannelManagerImpl) Start() {
m.closeWaiter.Add(1)
go func() {
defer m.closeWaiter.Done()
log.Info("DataNode ChannelManager start")
for {
select {
case opState := <-m.communicateCh:
m.handleOpState(opState)
case <-m.closeCh.CloseCh():
log.Info("DataNode ChannelManager exit")
return
}
}
}()
}
func (m *ChannelManagerImpl) handleOpState(opState *opState) {
m.mu.Lock()
defer m.mu.Unlock()
log := log.With(
zap.Int64("opID", opState.opID),
zap.String("channel", opState.channel),
zap.String("State", opState.state.String()),
)
switch opState.state {
case datapb.ChannelWatchState_WatchSuccess:
log.Info("Success to watch")
m.fgManager.AddFlowgraph(opState.fg)
case datapb.ChannelWatchState_WatchFailure:
log.Info("Fail to watch")
case datapb.ChannelWatchState_ReleaseSuccess:
log.Info("Success to release")
case datapb.ChannelWatchState_ReleaseFailure:
log.Info("Fail to release, add channel to abnormal lists")
m.abnormals.Insert(opState.opID, opState.channel)
}
m.finishOp(opState.opID, opState.channel)
}
func (m *ChannelManagerImpl) getOrCreateRunner(channel string) *opRunner {
runner, loaded := m.opRunners.GetOrInsert(channel, NewOpRunner(channel, m.pipelineParams, m.releaseFunc, executeWatch, m.communicateCh))
if !loaded {
runner.Start()
}
return runner
}
func (m *ChannelManagerImpl) finishOp(opID int64, channel string) {
if runner, loaded := m.opRunners.GetAndRemove(channel); loaded {
runner.FinishOp(opID)
runner.Close()
}
}
type opInfo struct {
tickler *util.Tickler
}
type opRunner struct {
channel string
pipelineParams *util.PipelineParams
releaseFunc releaseFunc
watchFunc watchFunc
guard sync.RWMutex
allOps map[typeutil.UniqueID]*opInfo // opID -> tickler
opsInQueue chan *datapb.ChannelWatchInfo
resultCh chan *opState
closeCh lifetime.SafeChan
closeWg sync.WaitGroup
}
func NewOpRunner(channel string, pipelineParams *util.PipelineParams, releaseF releaseFunc, watchF watchFunc, resultCh chan *opState) *opRunner {
return &opRunner{
channel: channel,
pipelineParams: pipelineParams,
releaseFunc: releaseF,
watchFunc: watchF,
opsInQueue: make(chan *datapb.ChannelWatchInfo, 10),
allOps: make(map[typeutil.UniqueID]*opInfo),
resultCh: resultCh,
closeCh: lifetime.NewSafeChan(),
}
}
func (r *opRunner) Start() {
r.closeWg.Add(1)
go func() {
defer r.closeWg.Done()
for {
select {
case info := <-r.opsInQueue:
r.NotifyState(r.Execute(info))
case <-r.closeCh.CloseCh():
return
}
}
}()
}
func (r *opRunner) FinishOp(opID typeutil.UniqueID) {
r.guard.Lock()
defer r.guard.Unlock()
delete(r.allOps, opID)
}
func (r *opRunner) Exist(opID typeutil.UniqueID) (progress int32, exists bool) {
r.guard.RLock()
defer r.guard.RUnlock()
info, ok := r.allOps[opID]
if !ok {
return -1, false
}
if info.tickler == nil {
return 0, true
}
return info.tickler.Progress(), true
}
func (r *opRunner) Enqueue(info *datapb.ChannelWatchInfo) error {
if info.GetState() != datapb.ChannelWatchState_ToWatch &&
info.GetState() != datapb.ChannelWatchState_ToRelease {
return errors.New("Invalid channel watch state")
}
r.guard.Lock()
defer r.guard.Unlock()
if _, ok := r.allOps[info.GetOpID()]; !ok {
r.opsInQueue <- info
r.allOps[info.GetOpID()] = &opInfo{}
}
return nil
}
func (r *opRunner) UnfinishedOpSize() int {
r.guard.RLock()
defer r.guard.RUnlock()
return len(r.allOps)
}
// Execute excutes channel operations, channel state is validated during enqueue
func (r *opRunner) Execute(info *datapb.ChannelWatchInfo) *opState {
log.Info("Start to execute channel operation",
zap.String("channel", info.GetVchan().GetChannelName()),
zap.Int64("opID", info.GetOpID()),
zap.String("state", info.GetState().String()),
)
if info.GetState() == datapb.ChannelWatchState_ToWatch {
return r.watchWithTimer(info)
}
// ToRelease state
return r.releaseWithTimer(r.releaseFunc, info.GetVchan().GetChannelName(), info.GetOpID())
}
func (r *opRunner) updateTickler(opID int64, tickler *util.Tickler) bool {
r.guard.Lock()
defer r.guard.Unlock()
opInfo, ok := r.allOps[opID]
if !ok {
return false
}
opInfo.tickler = tickler
return true
}
// watchWithTimer will return WatchFailure after WatchTimeoutInterval
func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState {
opState := &opState{
channel: info.GetVchan().GetChannelName(),
opID: info.GetOpID(),
}
log := log.With(zap.String("channel", opState.channel), zap.Int64("opID", opState.opID))
tickler := util.NewTickler()
ok := r.updateTickler(info.GetOpID(), tickler)
if !ok {
opState.state = datapb.ChannelWatchState_WatchFailure
return opState
}
var (
successSig = make(chan struct{}, 1)
finishWaiter sync.WaitGroup
)
watchTimeout := paramtable.Get().DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second)
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
startTimer := func(finishWg *sync.WaitGroup) {
defer finishWg.Done()
timer := time.NewTimer(watchTimeout)
defer timer.Stop()
log := log.With(zap.Duration("timeout", watchTimeout))
log.Info("Start timer for ToWatch operation")
for {
select {
case <-timer.C:
// watch timeout
tickler.Close()
cancel()
log.Info("Stop timer for ToWatch operation timeout")
return
case <-r.closeCh.CloseCh():
// runner closed from outside
tickler.Close()
cancel()
log.Info("Suspend ToWatch operation from outside of opRunner")
return
case <-tickler.GetProgressSig():
log.Info("Reset timer for tickler updated", zap.Int32("current progress", tickler.Progress()))
if !timer.Stop() {
select {
case <-timer.C:
default:
}
}
timer.Reset(watchTimeout)
case <-successSig:
// watch success
log.Info("Stop timer for ToWatch operation succeeded")
return
}
}
}
finishWaiter.Add(2)
go startTimer(&finishWaiter)
go func() {
defer finishWaiter.Done()
fg, err := r.watchFunc(ctx, r.pipelineParams, info, tickler)
if err != nil {
log.Warn("failed to watch channel", zap.Error(err))
opState.state = datapb.ChannelWatchState_WatchFailure
} else {
opState.state = datapb.ChannelWatchState_WatchSuccess
opState.fg = fg
successSig <- struct{}{}
}
}()
finishWaiter.Wait()
return opState
}
// releaseWithTimer will return ReleaseFailure after WatchTimeoutInterval
func (r *opRunner) releaseWithTimer(releaseFunc releaseFunc, channel string, opID typeutil.UniqueID) *opState {
opState := &opState{
channel: channel,
opID: opID,
}
var (
successSig = make(chan struct{}, 1)
finishWaiter sync.WaitGroup
)
log := log.With(zap.Int64("opID", opID), zap.String("channel", channel))
startTimer := func(finishWaiter *sync.WaitGroup) {
defer finishWaiter.Done()
releaseTimeout := paramtable.Get().DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second)
timer := time.NewTimer(releaseTimeout)
defer timer.Stop()
log := log.With(zap.Duration("timeout", releaseTimeout))
log.Info("Start ToRelease timer")
for {
select {
case <-timer.C:
log.Info("Stop timer for ToRelease operation timeout")
opState.state = datapb.ChannelWatchState_ReleaseFailure
return
case <-r.closeCh.CloseCh():
// runner closed from outside
log.Info("Stop timer for opRunner closed")
return
case <-successSig:
log.Info("Stop timer for ToRelease operation succeeded")
opState.state = datapb.ChannelWatchState_ReleaseSuccess
return
}
}
}
finishWaiter.Add(2)
go startTimer(&finishWaiter)
go func() {
defer finishWaiter.Done()
// TODO: failure should panic this DN, but we're not sure how
// to recover when releaseFunc stuck.
// Whenever we see a stuck, it's a bug need to be fixed.
// In case of the unknown behavior after the stuck of release,
// we'll mark this channel abnormal in this DN. This goroutine might never return.
//
// The channel can still be balanced into other DNs, but not on this one.
// ExclusiveConsumer error happens when the same DN subscribes the same pchannel twice.
releaseFunc(opState.channel)
successSig <- struct{}{}
}()
finishWaiter.Wait()
return opState
}
func (r *opRunner) NotifyState(state *opState) {
r.resultCh <- state
}
func (r *opRunner) Close() {
r.closeCh.Close()
r.closeWg.Wait()
}
type opState struct {
channel string
opID int64
state datapb.ChannelWatchState
fg *pipeline.DataSyncService
}
// executeWatch will always return, won't be stuck, either success or fail.
func executeWatch(ctx context.Context, pipelineParams *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler) (*pipeline.DataSyncService, error) {
dataSyncService, err := pipeline.NewDataSyncService(ctx, pipelineParams, info, tickler)
if err != nil {
return nil, err
}
dataSyncService.Start()
return dataSyncService, nil
}

View File

@ -1,353 +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 channel
import (
"context"
"os"
"testing"
"time"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/datanode/allocator"
"github.com/milvus-io/milvus/internal/flushcommon/broker"
"github.com/milvus-io/milvus/internal/flushcommon/pipeline"
"github.com/milvus-io/milvus/internal/flushcommon/syncmgr"
util2 "github.com/milvus-io/milvus/internal/flushcommon/util"
"github.com/milvus-io/milvus/internal/flushcommon/writebuffer"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/mq/msgdispatcher"
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
func TestMain(t *testing.M) {
paramtable.Init()
code := t.Run()
os.Exit(code)
}
func TestChannelManagerSuite(t *testing.T) {
suite.Run(t, new(ChannelManagerSuite))
}
func TestOpRunnerSuite(t *testing.T) {
suite.Run(t, new(OpRunnerSuite))
}
func (s *OpRunnerSuite) SetupTest() {
mockedBroker := broker.NewMockBroker(s.T())
mockedBroker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).
Return([]*datapb.SegmentInfo{}, nil).Maybe()
wbManager := writebuffer.NewMockBufferManager(s.T())
wbManager.EXPECT().
Register(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).
Return(nil).Maybe()
dispClient := msgdispatcher.NewMockClient(s.T())
dispClient.EXPECT().Register(mock.Anything, mock.Anything).
Return(make(chan *msgstream.MsgPack), nil).Maybe()
dispClient.EXPECT().Deregister(mock.Anything).Maybe()
s.pipelineParams = &util2.PipelineParams{
Ctx: context.TODO(),
Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 0}},
CheckpointUpdater: util2.NewChannelCheckpointUpdater(mockedBroker),
WriteBufferManager: wbManager,
Broker: mockedBroker,
DispClient: dispClient,
SyncMgr: syncmgr.NewMockSyncManager(s.T()),
Allocator: allocator.NewMockAllocator(s.T()),
}
}
func (s *OpRunnerSuite) TestWatchWithTimer() {
var (
channel string = "ch-1"
commuCh = make(chan *opState)
)
info := GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
mockReleaseFunc := func(channel string) {
log.Info("mock release func")
}
runner := NewOpRunner(channel, s.pipelineParams, mockReleaseFunc, executeWatch, commuCh)
err := runner.Enqueue(info)
s.Require().NoError(err)
opState := runner.watchWithTimer(info)
s.NotNil(opState.fg)
s.Equal(channel, opState.channel)
runner.FinishOp(100)
}
func (s *OpRunnerSuite) TestWatchTimeout() {
channel := "by-dev-rootcoord-dml-1000"
paramtable.Get().Save(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key, "0.000001")
defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key)
info := GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
sig := make(chan struct{})
commuCh := make(chan *opState)
mockReleaseFunc := func(channel string) { log.Info("mock release func") }
mockWatchFunc := func(ctx context.Context, param *util2.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util2.Tickler) (*pipeline.DataSyncService, error) {
<-ctx.Done()
sig <- struct{}{}
return nil, errors.New("timeout")
}
runner := NewOpRunner(channel, s.pipelineParams, mockReleaseFunc, mockWatchFunc, commuCh)
runner.Start()
defer runner.Close()
err := runner.Enqueue(info)
s.Require().NoError(err)
<-sig
opState := <-commuCh
s.Require().NotNil(opState)
s.Equal(info.GetOpID(), opState.opID)
s.Equal(datapb.ChannelWatchState_WatchFailure, opState.state)
}
type OpRunnerSuite struct {
suite.Suite
pipelineParams *util2.PipelineParams
}
type ChannelManagerSuite struct {
suite.Suite
pipelineParams *util2.PipelineParams
manager *ChannelManagerImpl
}
func (s *ChannelManagerSuite) SetupTest() {
factory := dependency.NewDefaultFactory(true)
wbManager := writebuffer.NewMockBufferManager(s.T())
wbManager.EXPECT().
Register(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).
Return(nil).Maybe()
wbManager.EXPECT().RemoveChannel(mock.Anything).Maybe()
mockedBroker := &broker.MockBroker{}
mockedBroker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).Return([]*datapb.SegmentInfo{}, nil).Maybe()
s.pipelineParams = &util2.PipelineParams{
Ctx: context.TODO(),
Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 0}},
WriteBufferManager: wbManager,
Broker: mockedBroker,
MsgStreamFactory: factory,
DispClient: msgdispatcher.NewClient(factory, typeutil.DataNodeRole, paramtable.GetNodeID()),
SyncMgr: syncmgr.NewMockSyncManager(s.T()),
Allocator: allocator.NewMockAllocator(s.T()),
}
s.manager = NewChannelManager(s.pipelineParams, pipeline.NewFlowgraphManager())
}
func (s *ChannelManagerSuite) TearDownTest() {
if s.manager != nil {
s.manager.Close()
}
}
func (s *ChannelManagerSuite) TestReleaseStuck() {
channel := "by-dev-rootcoord-dml-2"
s.manager.releaseFunc = func(channel string) {
time.Sleep(1 * time.Second)
}
info := GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
s.Require().Equal(0, s.manager.opRunners.Len())
err := s.manager.Submit(info)
s.Require().NoError(err)
opState := <-s.manager.communicateCh
s.Require().NotNil(opState)
s.manager.handleOpState(opState)
releaseInfo := GetWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease)
paramtable.Get().Save(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key, "0.1")
defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key)
err = s.manager.Submit(releaseInfo)
s.NoError(err)
opState = <-s.manager.communicateCh
s.Require().NotNil(opState)
s.Equal(datapb.ChannelWatchState_ReleaseFailure, opState.state)
s.manager.handleOpState(opState)
s.Equal(1, s.manager.abnormals.Len())
abchannel, ok := s.manager.abnormals.Get(releaseInfo.GetOpID())
s.True(ok)
s.Equal(channel, abchannel)
resp := s.manager.GetProgress(releaseInfo)
s.Equal(datapb.ChannelWatchState_ReleaseFailure, resp.GetState())
}
func (s *ChannelManagerSuite) TestSubmitIdempotent() {
channel := "by-dev-rootcoord-dml-1"
info := GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
s.Require().Equal(0, s.manager.opRunners.Len())
for i := 0; i < 10; i++ {
err := s.manager.Submit(info)
s.NoError(err)
}
s.Equal(1, s.manager.opRunners.Len())
s.True(s.manager.opRunners.Contain(channel))
runner, ok := s.manager.opRunners.Get(channel)
s.True(ok)
s.Equal(1, runner.UnfinishedOpSize())
}
func (s *ChannelManagerSuite) TestSubmitSkip() {
channel := "by-dev-rootcoord-dml-1"
info := GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
s.Require().Equal(0, s.manager.opRunners.Len())
err := s.manager.Submit(info)
s.NoError(err)
s.Equal(1, s.manager.opRunners.Len())
s.True(s.manager.opRunners.Contain(channel))
opState := <-s.manager.communicateCh
s.NotNil(opState)
s.Equal(datapb.ChannelWatchState_WatchSuccess, opState.state)
s.NotNil(opState.fg)
s.Equal(info.GetOpID(), opState.fg.GetOpID())
s.manager.handleOpState(opState)
err = s.manager.Submit(info)
s.NoError(err)
runner, ok := s.manager.opRunners.Get(channel)
s.False(ok)
s.Nil(runner)
}
func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() {
channel := "by-dev-rootcoord-dml-0"
stream, err := s.pipelineParams.MsgStreamFactory.NewTtMsgStream(context.Background())
s.NoError(err)
s.NotNil(stream)
stream.AsProducer(context.Background(), []string{channel})
// watch
info := GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch)
err = s.manager.Submit(info)
s.NoError(err)
// wait for result
opState := <-s.manager.communicateCh
s.NotNil(opState)
s.Equal(datapb.ChannelWatchState_WatchSuccess, opState.state)
s.NotNil(opState.fg)
s.Equal(info.GetOpID(), opState.fg.GetOpID())
resp := s.manager.GetProgress(info)
s.Equal(info.GetOpID(), resp.GetOpID())
s.Equal(datapb.ChannelWatchState_ToWatch, resp.GetState())
s.manager.handleOpState(opState)
s.Equal(1, s.manager.fgManager.GetFlowgraphCount())
s.False(s.manager.opRunners.Contain(info.GetVchan().GetChannelName()))
s.Equal(0, s.manager.opRunners.Len())
resp = s.manager.GetProgress(info)
s.Equal(info.GetOpID(), resp.GetOpID())
s.Equal(datapb.ChannelWatchState_WatchSuccess, resp.GetState())
// release
info = GetWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease)
err = s.manager.Submit(info)
s.NoError(err)
// wait for result
opState = <-s.manager.communicateCh
s.NotNil(opState)
s.Equal(datapb.ChannelWatchState_ReleaseSuccess, opState.state)
s.manager.handleOpState(opState)
resp = s.manager.GetProgress(info)
s.Equal(info.GetOpID(), resp.GetOpID())
s.Equal(datapb.ChannelWatchState_ReleaseSuccess, resp.GetState())
s.Equal(0, s.manager.fgManager.GetFlowgraphCount())
s.False(s.manager.opRunners.Contain(info.GetVchan().GetChannelName()))
s.Equal(0, s.manager.opRunners.Len())
err = s.manager.Submit(info)
s.NoError(err)
runner, ok := s.manager.opRunners.Get(channel)
s.False(ok)
s.Nil(runner)
}
func GetWatchInfoByOpID(opID typeutil.UniqueID, channel string, state datapb.ChannelWatchState) *datapb.ChannelWatchInfo {
return &datapb.ChannelWatchInfo{
OpID: opID,
State: state,
Vchan: &datapb.VchannelInfo{
CollectionID: 1,
ChannelName: channel,
},
Schema: &schemapb.CollectionSchema{
Name: "test_collection",
Fields: []*schemapb.FieldSchema{
{
FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64,
},
{
FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64,
},
{
FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true,
},
{
FieldID: 101, Name: "vector", DataType: schemapb.DataType_FloatVector,
TypeParams: []*commonpb.KeyValuePair{
{Key: common.DimKey, Value: "128"},
},
},
},
},
}
}

View File

@ -1,193 +0,0 @@
// Code generated by mockery v2.53.3. DO NOT EDIT.
package channel
import (
datapb "github.com/milvus-io/milvus/pkg/v2/proto/datapb"
mock "github.com/stretchr/testify/mock"
)
// MockChannelManager is an autogenerated mock type for the ChannelManager type
type MockChannelManager struct {
mock.Mock
}
type MockChannelManager_Expecter struct {
mock *mock.Mock
}
func (_m *MockChannelManager) EXPECT() *MockChannelManager_Expecter {
return &MockChannelManager_Expecter{mock: &_m.Mock}
}
// Close provides a mock function with no fields
func (_m *MockChannelManager) Close() {
_m.Called()
}
// MockChannelManager_Close_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Close'
type MockChannelManager_Close_Call struct {
*mock.Call
}
// Close is a helper method to define mock.On call
func (_e *MockChannelManager_Expecter) Close() *MockChannelManager_Close_Call {
return &MockChannelManager_Close_Call{Call: _e.mock.On("Close")}
}
func (_c *MockChannelManager_Close_Call) Run(run func()) *MockChannelManager_Close_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockChannelManager_Close_Call) Return() *MockChannelManager_Close_Call {
_c.Call.Return()
return _c
}
func (_c *MockChannelManager_Close_Call) RunAndReturn(run func()) *MockChannelManager_Close_Call {
_c.Run(run)
return _c
}
// GetProgress provides a mock function with given fields: info
func (_m *MockChannelManager) GetProgress(info *datapb.ChannelWatchInfo) *datapb.ChannelOperationProgressResponse {
ret := _m.Called(info)
if len(ret) == 0 {
panic("no return value specified for GetProgress")
}
var r0 *datapb.ChannelOperationProgressResponse
if rf, ok := ret.Get(0).(func(*datapb.ChannelWatchInfo) *datapb.ChannelOperationProgressResponse); ok {
r0 = rf(info)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*datapb.ChannelOperationProgressResponse)
}
}
return r0
}
// MockChannelManager_GetProgress_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetProgress'
type MockChannelManager_GetProgress_Call struct {
*mock.Call
}
// GetProgress is a helper method to define mock.On call
// - info *datapb.ChannelWatchInfo
func (_e *MockChannelManager_Expecter) GetProgress(info interface{}) *MockChannelManager_GetProgress_Call {
return &MockChannelManager_GetProgress_Call{Call: _e.mock.On("GetProgress", info)}
}
func (_c *MockChannelManager_GetProgress_Call) Run(run func(info *datapb.ChannelWatchInfo)) *MockChannelManager_GetProgress_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(*datapb.ChannelWatchInfo))
})
return _c
}
func (_c *MockChannelManager_GetProgress_Call) Return(_a0 *datapb.ChannelOperationProgressResponse) *MockChannelManager_GetProgress_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockChannelManager_GetProgress_Call) RunAndReturn(run func(*datapb.ChannelWatchInfo) *datapb.ChannelOperationProgressResponse) *MockChannelManager_GetProgress_Call {
_c.Call.Return(run)
return _c
}
// Start provides a mock function with no fields
func (_m *MockChannelManager) Start() {
_m.Called()
}
// MockChannelManager_Start_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Start'
type MockChannelManager_Start_Call struct {
*mock.Call
}
// Start is a helper method to define mock.On call
func (_e *MockChannelManager_Expecter) Start() *MockChannelManager_Start_Call {
return &MockChannelManager_Start_Call{Call: _e.mock.On("Start")}
}
func (_c *MockChannelManager_Start_Call) Run(run func()) *MockChannelManager_Start_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockChannelManager_Start_Call) Return() *MockChannelManager_Start_Call {
_c.Call.Return()
return _c
}
func (_c *MockChannelManager_Start_Call) RunAndReturn(run func()) *MockChannelManager_Start_Call {
_c.Run(run)
return _c
}
// Submit provides a mock function with given fields: info
func (_m *MockChannelManager) Submit(info *datapb.ChannelWatchInfo) error {
ret := _m.Called(info)
if len(ret) == 0 {
panic("no return value specified for Submit")
}
var r0 error
if rf, ok := ret.Get(0).(func(*datapb.ChannelWatchInfo) error); ok {
r0 = rf(info)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockChannelManager_Submit_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Submit'
type MockChannelManager_Submit_Call struct {
*mock.Call
}
// Submit is a helper method to define mock.On call
// - info *datapb.ChannelWatchInfo
func (_e *MockChannelManager_Expecter) Submit(info interface{}) *MockChannelManager_Submit_Call {
return &MockChannelManager_Submit_Call{Call: _e.mock.On("Submit", info)}
}
func (_c *MockChannelManager_Submit_Call) Run(run func(info *datapb.ChannelWatchInfo)) *MockChannelManager_Submit_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(*datapb.ChannelWatchInfo))
})
return _c
}
func (_c *MockChannelManager_Submit_Call) Return(_a0 error) *MockChannelManager_Submit_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockChannelManager_Submit_Call) RunAndReturn(run func(*datapb.ChannelWatchInfo) error) *MockChannelManager_Submit_Call {
_c.Call.Return(run)
return _c
}
// NewMockChannelManager creates a new instance of MockChannelManager. 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 NewMockChannelManager(t interface {
mock.TestingT
Cleanup(func())
}) *MockChannelManager {
mock := &MockChannelManager{}
mock.Mock.Test(t)
t.Cleanup(func() { mock.AssertExpectations(t) })
return mock
}

View File

@ -269,7 +269,7 @@ func (s *ClusteringCompactionTaskStorageV2Suite) initStorageV2Segments(rows int,
metacache.UpdateNumOfRows(int64(rows))(seg)
mc := metacache.NewMockMetaCache(s.T())
mc.EXPECT().Collection().Return(CollectionID).Maybe()
mc.EXPECT().Schema().Return(genCollectionSchema()).Maybe()
mc.EXPECT().GetSchema(mock.Anything).Return(genCollectionSchema()).Maybe()
mc.EXPECT().GetSegmentByID(segmentID).Return(seg, true).Maybe()
mc.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg}).Maybe()
mc.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Run(func(action metacache.SegmentAction, filters ...metacache.SegmentFilter) {

View File

@ -312,7 +312,7 @@ func (s *MixCompactionTaskStorageV2Suite) initStorageV2Segments(rows int, seed i
metacache.UpdateNumOfRows(1000)(seg)
mc := metacache.NewMockMetaCache(s.T())
mc.EXPECT().Collection().Return(CollectionID).Maybe()
mc.EXPECT().Schema().Return(s.meta.Schema).Maybe()
mc.EXPECT().GetSchema(mock.Anything).Return(s.meta.Schema).Maybe()
mc.EXPECT().GetSegmentByID(seed).Return(seg, true).Maybe()
mc.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg}).Maybe()
mc.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Run(func(action metacache.SegmentAction, filters ...metacache.SegmentFilter) {

View File

@ -98,7 +98,7 @@ func NewSyncTask(ctx context.Context,
task := syncmgr.NewSyncTask().
WithAllocator(allocator).
WithMetaCache(metaCache).
WithSchema(metaCache.Schema()). // TODO specify import schema if needed
WithSchema(metaCache.GetSchema(0)). // TODO specify import schema if needed
WithSyncPack(syncPack).
WithStorageConfig(storageConfig)
return task, nil

View File

@ -23,7 +23,6 @@ import (
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/v2/log"
@ -32,10 +31,10 @@ import (
//go:generate mockery --name=MetaCache --structname=MockMetaCache --output=./ --filename=mock_meta_cache.go --with-expecter --inpackage
type MetaCache interface {
SchemaManager
// Collection returns collection id of metacache.
Collection() int64
// Schema returns collection schema.
Schema() *schemapb.CollectionSchema
// AddSegment adds a segment from segment info.
AddSegment(segInfo *datapb.SegmentInfo, pkFactory PkStatsFactory, bmFactory BM25StatsFactory, actions ...SegmentAction)
// UpdateSegments applies action to segment(s) satisfy the provided filters.
@ -54,8 +53,6 @@ type MetaCache interface {
DetectMissingSegments(segments map[int64]struct{}) []int64
// UpdateSegmentView updates the segments BF from datacoord view.
UpdateSegmentView(partitionID int64, newSegments []*datapb.SyncSegmentInfo, newSegmentsBF []*pkoracle.BloomFilterSet, allSegments map[int64]struct{})
// UpdateSchema update the latest collection schema
UpdateSchema(updatedSchema *schemapb.CollectionSchema, version uint64)
}
var _ MetaCache = (*metaCacheImpl)(nil)
@ -74,23 +71,33 @@ func NewBM25StatsFactory(vchannel *datapb.SegmentInfo) *SegmentBM25Stats {
}
type metaCacheImpl struct {
SchemaManager
collectionID int64
vChannelName string
schema *schemapb.CollectionSchema
mu sync.RWMutex
segmentInfos map[int64]*SegmentInfo
stateSegments map[commonpb.SegmentState]map[int64]*SegmentInfo
}
func NewMetaCache(info *datapb.ChannelWatchInfo, pkFactory PkStatsFactory, bmFactor BM25StatsFactory) MetaCache {
func NewMetaCache(
info *datapb.ChannelWatchInfo,
pkFactory PkStatsFactory,
bmFactor BM25StatsFactory,
schemaManager ...SchemaManager,
) MetaCache {
vchannel := info.GetVchan()
var m SchemaManager = newVersionlessSchemaManager(info.GetSchema())
if len(schemaManager) > 0 && schemaManager[0] != nil {
m = schemaManager[0]
}
cache := &metaCacheImpl{
SchemaManager: m,
collectionID: vchannel.GetCollectionID(),
vChannelName: vchannel.GetChannelName(),
segmentInfos: make(map[int64]*SegmentInfo),
stateSegments: make(map[commonpb.SegmentState]map[int64]*SegmentInfo),
schema: info.GetSchema(),
}
for _, state := range []commonpb.SegmentState{
@ -125,11 +132,6 @@ func (c *metaCacheImpl) Collection() int64 {
return c.collectionID
}
// Schema returns collection schema.
func (c *metaCacheImpl) Schema() *schemapb.CollectionSchema {
return c.schema
}
// AddSegment adds a segment from segment info.
func (c *metaCacheImpl) AddSegment(segInfo *datapb.SegmentInfo, pkFactory PkStatsFactory, bmFactory BM25StatsFactory, actions ...SegmentAction) {
segment := NewSegmentInfo(segInfo, pkFactory(segInfo), bmFactory(segInfo))
@ -318,7 +320,3 @@ func (c *metaCacheImpl) UpdateSegmentView(partitionID int64,
}
}
}
func (c *metaCacheImpl) UpdateSchema(updatedSchema *schemapb.CollectionSchema, _ uint64) {
c.schema = updatedSchema
}

View File

@ -101,7 +101,7 @@ func (s *MetaCacheSuite) SetupTest() {
func (s *MetaCacheSuite) TestMetaInfo() {
s.Equal(s.collectionID, s.cache.Collection())
s.Equal(s.collSchema, s.cache.Schema())
s.Equal(s.collSchema, s.cache.GetSchema(0))
}
func (s *MetaCacheSuite) TestAddSegment() {

View File

@ -169,6 +169,54 @@ func (_c *MockMetaCache_DetectMissingSegments_Call) RunAndReturn(run func(map[in
return _c
}
// GetSchema provides a mock function with given fields: timetick
func (_m *MockMetaCache) GetSchema(timetick uint64) *schemapb.CollectionSchema {
ret := _m.Called(timetick)
if len(ret) == 0 {
panic("no return value specified for GetSchema")
}
var r0 *schemapb.CollectionSchema
if rf, ok := ret.Get(0).(func(uint64) *schemapb.CollectionSchema); ok {
r0 = rf(timetick)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*schemapb.CollectionSchema)
}
}
return r0
}
// MockMetaCache_GetSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetSchema'
type MockMetaCache_GetSchema_Call struct {
*mock.Call
}
// GetSchema is a helper method to define mock.On call
// - timetick uint64
func (_e *MockMetaCache_Expecter) GetSchema(timetick interface{}) *MockMetaCache_GetSchema_Call {
return &MockMetaCache_GetSchema_Call{Call: _e.mock.On("GetSchema", timetick)}
}
func (_c *MockMetaCache_GetSchema_Call) Run(run func(timetick uint64)) *MockMetaCache_GetSchema_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(uint64))
})
return _c
}
func (_c *MockMetaCache_GetSchema_Call) Return(_a0 *schemapb.CollectionSchema) *MockMetaCache_GetSchema_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockMetaCache_GetSchema_Call) RunAndReturn(run func(uint64) *schemapb.CollectionSchema) *MockMetaCache_GetSchema_Call {
_c.Call.Return(run)
return _c
}
// GetSegmentByID provides a mock function with given fields: id, filters
func (_m *MockMetaCache) GetSegmentByID(id int64, filters ...SegmentFilter) (*SegmentInfo, bool) {
_va := make([]interface{}, len(filters))
@ -498,87 +546,6 @@ func (_c *MockMetaCache_RemoveSegments_Call) RunAndReturn(run func(...SegmentFil
return _c
}
// Schema provides a mock function with no fields
func (_m *MockMetaCache) Schema() *schemapb.CollectionSchema {
ret := _m.Called()
if len(ret) == 0 {
panic("no return value specified for Schema")
}
var r0 *schemapb.CollectionSchema
if rf, ok := ret.Get(0).(func() *schemapb.CollectionSchema); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*schemapb.CollectionSchema)
}
}
return r0
}
// MockMetaCache_Schema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Schema'
type MockMetaCache_Schema_Call struct {
*mock.Call
}
// Schema is a helper method to define mock.On call
func (_e *MockMetaCache_Expecter) Schema() *MockMetaCache_Schema_Call {
return &MockMetaCache_Schema_Call{Call: _e.mock.On("Schema")}
}
func (_c *MockMetaCache_Schema_Call) Run(run func()) *MockMetaCache_Schema_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockMetaCache_Schema_Call) Return(_a0 *schemapb.CollectionSchema) *MockMetaCache_Schema_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockMetaCache_Schema_Call) RunAndReturn(run func() *schemapb.CollectionSchema) *MockMetaCache_Schema_Call {
_c.Call.Return(run)
return _c
}
// UpdateSchema provides a mock function with given fields: updatedSchema, version
func (_m *MockMetaCache) UpdateSchema(updatedSchema *schemapb.CollectionSchema, version uint64) {
_m.Called(updatedSchema, version)
}
// MockMetaCache_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema'
type MockMetaCache_UpdateSchema_Call struct {
*mock.Call
}
// UpdateSchema is a helper method to define mock.On call
// - updatedSchema *schemapb.CollectionSchema
// - version uint64
func (_e *MockMetaCache_Expecter) UpdateSchema(updatedSchema interface{}, version interface{}) *MockMetaCache_UpdateSchema_Call {
return &MockMetaCache_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema", updatedSchema, version)}
}
func (_c *MockMetaCache_UpdateSchema_Call) Run(run func(updatedSchema *schemapb.CollectionSchema, version uint64)) *MockMetaCache_UpdateSchema_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(*schemapb.CollectionSchema), args[1].(uint64))
})
return _c
}
func (_c *MockMetaCache_UpdateSchema_Call) Return() *MockMetaCache_UpdateSchema_Call {
_c.Call.Return()
return _c
}
func (_c *MockMetaCache_UpdateSchema_Call) RunAndReturn(run func(*schemapb.CollectionSchema, uint64)) *MockMetaCache_UpdateSchema_Call {
_c.Run(run)
return _c
}
// UpdateSegmentView provides a mock function with given fields: partitionID, newSegments, newSegmentsBF, allSegments
func (_m *MockMetaCache) UpdateSegmentView(partitionID int64, newSegments []*datapb.SyncSegmentInfo, newSegmentsBF []*pkoracle.BloomFilterSet, allSegments map[int64]struct{}) {
_m.Called(partitionID, newSegments, newSegmentsBF, allSegments)

View File

@ -0,0 +1,22 @@
package metacache
import "github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
// SchemaManager is a manager for collection schema.
type SchemaManager interface {
// GetSchema returns the schema at the given timetick
GetSchema(timetick uint64) *schemapb.CollectionSchema
}
// newVersionlessSchemaManager creates a new versionless schema manager.
func newVersionlessSchemaManager(schema *schemapb.CollectionSchema) SchemaManager {
return &versionlessSchemaManager{schema: schema}
}
type versionlessSchemaManager struct {
schema *schemapb.CollectionSchema
}
func (m *versionlessSchemaManager) GetSchema(timetick uint64) *schemapb.CollectionSchema {
return m.schema
}

View File

@ -0,0 +1,18 @@
package metacache
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
)
func TestVersionlessSchemaManager(t *testing.T) {
schema := &schemapb.CollectionSchema{
Name: "test",
}
manager := newVersionlessSchemaManager(schema)
assert.Equal(t, schema, manager.GetSchema(0))
}

View File

@ -23,7 +23,6 @@ import (
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/compaction"
"github.com/milvus-io/milvus/internal/datanode/compactor"
"github.com/milvus-io/milvus/internal/flushcommon/broker"
@ -137,15 +136,15 @@ func (dsService *DataSyncService) GetMetaCache() metacache.MetaCache {
}
func getMetaCacheForStreaming(initCtx context.Context, params *util.PipelineParams, info *datapb.ChannelWatchInfo, unflushed, flushed []*datapb.SegmentInfo) (metacache.MetaCache, error) {
return initMetaCache(initCtx, params.ChunkManager, info, nil, unflushed, flushed)
return initMetaCache(initCtx, params.ChunkManager, info, nil, unflushed, flushed, params.SchemaManager)
}
func getMetaCacheWithTickler(initCtx context.Context, params *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler, unflushed, flushed []*datapb.SegmentInfo) (metacache.MetaCache, error) {
tickler.SetTotal(int32(len(unflushed) + len(flushed)))
return initMetaCache(initCtx, params.ChunkManager, info, tickler, unflushed, flushed)
return initMetaCache(initCtx, params.ChunkManager, info, tickler, unflushed, flushed, params.SchemaManager)
}
func initMetaCache(initCtx context.Context, chunkManager storage.ChunkManager, info *datapb.ChannelWatchInfo, tickler interface{ Inc() }, unflushed, flushed []*datapb.SegmentInfo) (metacache.MetaCache, error) {
func initMetaCache(initCtx context.Context, chunkManager storage.ChunkManager, info *datapb.ChannelWatchInfo, tickler interface{ Inc() }, unflushed, flushed []*datapb.SegmentInfo, schemaManager metacache.SchemaManager) (metacache.MetaCache, error) {
// tickler will update addSegment progress to watchInfo
futures := make([]*conc.Future[any], 0, len(unflushed)+len(flushed))
// segmentPks := typeutil.NewConcurrentMap[int64, []*storage.PkStatistics]()
@ -174,7 +173,7 @@ func initMetaCache(initCtx context.Context, chunkManager storage.ChunkManager, i
}
if segType == "growing" && len(segment.GetBm25Statslogs()) > 0 {
bm25stats, err := compaction.LoadBM25Stats(initCtx, chunkManager, info.GetSchema(), segment.GetID(), segment.GetBm25Statslogs())
bm25stats, err := compaction.LoadBM25Stats(initCtx, chunkManager, segment.GetID(), segment.GetBm25Statslogs())
if err != nil {
return nil, err
}
@ -217,7 +216,7 @@ func initMetaCache(initCtx context.Context, chunkManager storage.ChunkManager, i
return segmentStats
}
// return channel, nil
metacache := metacache.NewMetaCache(info, pkStatsFactory, bm25StatsFactor)
metacache := metacache.NewMetaCache(info, pkStatsFactory, bm25StatsFactor, schemaManager)
return metacache, nil
}
@ -417,13 +416,12 @@ func NewEmptyStreamingNodeDataSyncService(
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,
Schema: pipelineParams.SchemaManager.GetSchema(0), // use the latest schema.
}
metaCache, err := getMetaCacheForStreaming(initCtx, pipelineParams, watchInfo, make([]*datapb.SegmentInfo, 0), make([]*datapb.SegmentInfo, 0))
if err != nil {

View File

@ -282,27 +282,6 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
} else {
logger.Info("handle manual flush message success")
}
case commonpb.MsgType_AddCollectionField:
schemaMsg := msg.(*adaptor.SchemaChangeMessageBody)
header := schemaMsg.SchemaChangeMessage.Header()
if header.GetCollectionId() != ddn.collectionID {
continue
}
logger := log.With(
zap.String("vchannel", ddn.Name()),
zap.Int32("msgType", int32(msg.Type())),
zap.Uint64("timetick", schemaMsg.SchemaChangeMessage.TimeTick()),
zap.Int64s("segmentIDs", schemaMsg.SchemaChangeMessage.Header().FlushedSegmentIds),
)
logger.Info("receive schema change message")
body, err := schemaMsg.SchemaChangeMessage.Body()
if err != nil {
logger.Warn("failed to unmarshal schema change message body", zap.Error(err))
continue
}
fgMsg.updatedSchema = body.GetSchema()
fgMsg.schemaVersion = schemaMsg.BeginTs()
ddn.msgHandler.HandleSchemaChange(ddn.ctx, schemaMsg.SchemaChangeMessage)
}
}

View File

@ -58,7 +58,7 @@ func newEmbeddingNode(channelName string, metaCache metacache.MetaCache) (*embed
functionRunners: make(map[int64]function.FunctionRunner),
}
schema := metaCache.Schema()
schema := metaCache.GetSchema(0)
for _, field := range schema.GetFields() {
if field.GetIsPrimaryKey() {
@ -155,7 +155,7 @@ func (eNode *embeddingNode) Operate(in []Msg) []Msg {
return []Msg{fgMsg}
}
insertData, err := writebuffer.PrepareInsert(eNode.metaCache.Schema(), eNode.pkField, fgMsg.InsertMessages)
insertData, err := writebuffer.PrepareInsert(eNode.metaCache.GetSchema(fgMsg.TimeTick()), eNode.pkField, fgMsg.InsertMessages)
if err != nil {
log.Error("failed to prepare insert data", zap.Error(err))
panic(err)

View File

@ -20,6 +20,7 @@ import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
@ -68,7 +69,7 @@ func TestEmbeddingNode_BM25_Operator(t *testing.T) {
}
metaCache := metacache.NewMockMetaCache(t)
metaCache.EXPECT().Schema().Return(collSchema)
metaCache.EXPECT().GetSchema(mock.Anything).Return(collSchema)
t.Run("normal case", func(t *testing.T) {
node, err := newEmbeddingNode("test-channel", metaCache)

View File

@ -18,7 +18,6 @@ package pipeline
import (
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/flushcommon/util"
"github.com/milvus-io/milvus/internal/flushcommon/writebuffer"
"github.com/milvus-io/milvus/internal/storage"
@ -61,9 +60,6 @@ type FlowGraphMsg struct {
segmentsToSync []typeutil.UniqueID
dropCollection bool
dropPartitions []typeutil.UniqueID
updatedSchema *schemapb.CollectionSchema
schemaVersion uint64
}
func (fgMsg *FlowGraphMsg) TimeTick() typeutil.Timestamp {

View File

@ -82,7 +82,7 @@ func (wNode *writeNode) Operate(in []Msg) []Msg {
start, end := fgMsg.StartPositions[0], fgMsg.EndPositions[0]
if fgMsg.InsertData == nil {
insertData, err := writebuffer.PrepareInsert(wNode.metacache.Schema(), wNode.pkField, fgMsg.InsertMessages)
insertData, err := writebuffer.PrepareInsert(wNode.metacache.GetSchema(fgMsg.TimeTick()), wNode.pkField, fgMsg.InsertMessages)
if err != nil {
log.Error("failed to prepare data", zap.Error(err))
panic(err)
@ -114,11 +114,6 @@ func (wNode *writeNode) Operate(in []Msg) []Msg {
wNode.updater.Update(wNode.channelName, end.GetTimestamp(), stats)
}
// update schema after all data processed
if fgMsg.updatedSchema != nil {
wNode.metacache.UpdateSchema(fgMsg.updatedSchema, fgMsg.schemaVersion)
}
res := FlowGraphMsg{
TimeRange: fgMsg.TimeRange,
StartPositions: fgMsg.StartPositions,
@ -148,7 +143,8 @@ func newWriteNode(
baseNode.SetMaxQueueLength(paramtable.Get().DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32())
baseNode.SetMaxParallelism(paramtable.Get().DataNodeCfg.FlowGraphMaxParallelism.GetAsInt32())
collSchema := config.metacache.Schema()
// pkfield is a immutable property of the collection, so we can get it from any schema
collSchema := config.metacache.GetSchema(0)
pkField, err := typeutil.GetPrimaryFieldSchema(collSchema)
if err != nil {
return nil, err

View File

@ -272,7 +272,7 @@ func (bw *BulkPackWriter) writeBM25Stasts(ctx context.Context, pack *SyncPack) (
if pack.isFlush {
if pack.level != datapb.SegmentLevel_L0 {
if hasBM25Function(bw.metaCache.Schema()) {
if hasBM25Function(bw.metaCache.GetSchema(pack.tsFrom)) {
mergedBM25Blob, err := serializer.serializeMergedBM25Stats(pack)
if err != nil {
return nil, err

View File

@ -96,7 +96,7 @@ func TestBulkPackWriter_Write(t *testing.T) {
mc := metacache.NewMockMetaCache(t)
mc.EXPECT().Collection().Return(collectionID).Maybe()
mc.EXPECT().Schema().Return(schema).Maybe()
mc.EXPECT().GetSchema(mock.Anything).Return(schema).Maybe()
mc.EXPECT().GetSegmentByID(segmentID).Return(seg, true).Maybe()
mc.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg}).Maybe()
mc.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Run(func(action metacache.SegmentAction, filters ...metacache.SegmentFilter) {

View File

@ -105,7 +105,7 @@ func (s *PackWriterV2Suite) TestPackWriterV2_Write() {
metacache.UpdateNumOfRows(1000)(seg)
mc := metacache.NewMockMetaCache(s.T())
mc.EXPECT().Collection().Return(collectionID).Maybe()
mc.EXPECT().Schema().Return(s.schema).Maybe()
mc.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe()
mc.EXPECT().GetSegmentByID(segmentID).Return(seg, true).Maybe()
mc.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg}).Maybe()
mc.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Run(func(action metacache.SegmentAction, filters ...metacache.SegmentFilter) {
@ -137,7 +137,7 @@ func (s *PackWriterV2Suite) TestWriteEmptyInsertData() {
segmentID := int64(789)
channelName := fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", collectionID)
mc := metacache.NewMockMetaCache(s.T())
mc.EXPECT().Schema().Return(s.schema).Maybe()
mc.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe()
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName)
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil)
@ -160,7 +160,7 @@ func (s *PackWriterV2Suite) TestNoPkField() {
segmentID := int64(789)
channelName := fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", collectionID)
mc := metacache.NewMockMetaCache(s.T())
mc.EXPECT().Schema().Return(s.schema).Maybe()
mc.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe()
buf, _ := storage.NewInsertData(s.schema)
data := make(map[storage.FieldID]any)
@ -183,7 +183,7 @@ func (s *PackWriterV2Suite) TestAllocIDExhausedError() {
channelName := fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", collectionID)
rows := 10
mc := metacache.NewMockMetaCache(s.T())
mc.EXPECT().Schema().Return(s.schema).Maybe()
mc.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe()
pack := new(SyncPack).WithCollectionID(collectionID).WithPartitionID(partitionID).WithSegmentID(segmentID).WithChannelName(channelName).WithInsertData(genInsertData(rows, s.schema))
bw := NewBulkPackWriterV2(mc, s.schema, s.cm, s.logIDAlloc, packed.DefaultWriteBufferSize, 0, nil)
@ -199,7 +199,7 @@ func (s *PackWriterV2Suite) TestWriteInsertDataError() {
segmentID := int64(789)
channelName := fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", collectionID)
mc := metacache.NewMockMetaCache(s.T())
mc.EXPECT().Schema().Return(s.schema).Maybe()
mc.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe()
buf, _ := storage.NewInsertData(s.schema)
data := make(map[storage.FieldID]any)

View File

@ -24,6 +24,7 @@ import (
"time"
"github.com/samber/lo"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
@ -91,7 +92,7 @@ func (s *StorageV1SerializerSuite) SetupSuite() {
}
func (s *StorageV1SerializerSuite) SetupTest() {
s.mockCache.EXPECT().Schema().Return(s.schema).Maybe()
s.mockCache.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe()
var err error
s.serializer, err = NewStorageSerializer(s.mockCache, s.schema)

View File

@ -109,7 +109,7 @@ func (s *SyncTaskSuite) SetupTest() {
s.broker = broker.NewMockBroker(s.T())
s.metacache = metacache.NewMockMetaCache(s.T())
s.metacache.EXPECT().Collection().Return(s.collectionID).Maybe()
s.metacache.EXPECT().Schema().Return(s.schema).Maybe()
s.metacache.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe()
initcore.InitLocalArrowFileSystem("/tmp")
}
@ -364,7 +364,7 @@ func (s *SyncTaskSuite) TestRunError() {
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(seg, true)
s.metacache.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg})
s.metacache.EXPECT().Collection().Return(s.collectionID).Maybe()
s.metacache.EXPECT().Schema().Return(s.schema).Maybe()
s.metacache.EXPECT().GetSchema(mock.Anything).Return(s.schema).Maybe()
s.Run("allocate_id_fail", func() {
mockAllocator := allocator.NewMockAllocator(s.T())

View File

@ -25,6 +25,7 @@ import (
"github.com/milvus-io/milvus/internal/allocator"
compactor "github.com/milvus-io/milvus/internal/datanode/compactor"
"github.com/milvus-io/milvus/internal/flushcommon/broker"
"github.com/milvus-io/milvus/internal/flushcommon/metacache"
"github.com/milvus-io/milvus/internal/flushcommon/syncmgr"
"github.com/milvus-io/milvus/internal/flushcommon/writebuffer"
"github.com/milvus-io/milvus/internal/storage"
@ -49,6 +50,7 @@ type PipelineParams struct {
CheckpointUpdater *ChannelCheckpointUpdater
Allocator allocator.Interface
MsgHandler MsgHandler
SchemaManager metacache.SchemaManager
}
// TimeRange is a range of timestamp contains the min-timestamp and max-timestamp

View File

@ -208,7 +208,7 @@ func (wb *l0WriteBuffer) BufferData(insertData []*InsertData, deleteMsgs []*msgs
// bufferInsert function InsertMsg into bufferred InsertData and returns primary key field data for future usage.
func (wb *l0WriteBuffer) bufferInsert(inData *InsertData, startPos, endPos *msgpb.MsgPosition) error {
wb.CreateNewGrowingSegment(inData.partitionID, inData.segmentID, startPos)
segBuf := wb.getOrCreateBuffer(inData.segmentID)
segBuf := wb.getOrCreateBuffer(inData.segmentID, startPos.GetTimestamp())
totalMemSize := segBuf.insertBuffer.Buffer(inData, startPos, endPos)
wb.metaCache.UpdateSegments(metacache.SegmentActions(

View File

@ -167,7 +167,7 @@ func (s *L0WriteBufferSuite) composeDeleteMsg(pks []storage.PrimaryKey) *msgstre
func (s *L0WriteBufferSuite) SetupTest() {
s.syncMgr = syncmgr.NewMockSyncManager(s.T())
s.metacache = metacache.NewMockMetaCache(s.T())
s.metacache.EXPECT().Schema().Return(s.collSchema).Maybe()
s.metacache.EXPECT().GetSchema(mock.Anything).Return(s.collSchema).Maybe()
s.metacache.EXPECT().Collection().Return(s.collID).Maybe()
s.allocator = allocator.NewMockGIDAllocator()
s.allocator.AllocOneF = func() (int64, error) { return int64(tsoutil.ComposeTSByTime(time.Now(), 0)), nil }

View File

@ -61,7 +61,7 @@ func (s *ManagerSuite) SetupTest() {
s.syncMgr = syncmgr.NewMockSyncManager(s.T())
s.metacache = metacache.NewMockMetaCache(s.T())
s.metacache.EXPECT().Collection().Return(s.collID).Maybe()
s.metacache.EXPECT().Schema().Return(s.collSchema).Maybe()
s.metacache.EXPECT().GetSchema(mock.Anything).Return(s.collSchema).Maybe()
s.allocator = allocator.NewMockAllocator(s.T())
mgr := NewManager(s.syncMgr)

View File

@ -144,7 +144,7 @@ func newWriteBufferBase(channel string, metacache metacache.MetaCache, syncMgr s
flushTsPolicy := GetFlushTsPolicy(flushTs, metacache)
option.syncPolicies = append(option.syncPolicies, flushTsPolicy)
schema := metacache.Schema()
schema := metacache.GetSchema(0)
estSize, err := typeutil.EstimateSizePerRecord(schema)
if err != nil {
return nil, err
@ -356,11 +356,11 @@ func (wb *writeBufferBase) getSegmentsToSync(ts typeutil.Timestamp, policies ...
return segments.Collect()
}
func (wb *writeBufferBase) getOrCreateBuffer(segmentID int64) *segmentBuffer {
func (wb *writeBufferBase) getOrCreateBuffer(segmentID int64, timetick uint64) *segmentBuffer {
buffer, ok := wb.buffers[segmentID]
if !ok {
var err error
buffer, err = newSegmentBuffer(segmentID, wb.metaCache.Schema())
buffer, err = newSegmentBuffer(segmentID, wb.metaCache.GetSchema(timetick))
if err != nil {
// TODO avoid panic here
panic(err)
@ -525,7 +525,7 @@ func (wb *writeBufferBase) CreateNewGrowingSegment(partitionID int64, segmentID
// bufferDelete buffers DeleteMsg into DeleteData.
func (wb *writeBufferBase) bufferDelete(segmentID int64, pks []storage.PrimaryKey, tss []typeutil.Timestamp, startPos, endPos *msgpb.MsgPosition) {
segBuf := wb.getOrCreateBuffer(segmentID)
segBuf := wb.getOrCreateBuffer(segmentID, tss[0])
bufSize := segBuf.deltaBuffer.Buffer(pks, tss, startPos, endPos)
metrics.DataNodeFlowGraphBufferDataSize.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), fmt.Sprint(wb.collectionID)).Add(float64(bufSize))
}

View File

@ -48,7 +48,7 @@ func (s *WriteBufferSuite) SetupSuite() {
func (s *WriteBufferSuite) SetupTest() {
s.syncMgr = syncmgr.NewMockSyncManager(s.T())
s.metacache = metacache.NewMockMetaCache(s.T())
s.metacache.EXPECT().Schema().Return(s.collSchema).Maybe()
s.metacache.EXPECT().GetSchema(mock.Anything).Return(s.collSchema).Maybe()
s.metacache.EXPECT().Collection().Return(s.collID).Maybe()
var err error
s.wb, err = newWriteBufferBase(s.channelName, s.metacache, s.syncMgr, &writeBufferOption{
@ -64,7 +64,7 @@ func (s *WriteBufferSuite) TestHasSegment() {
s.False(s.wb.HasSegment(segmentID))
s.wb.getOrCreateBuffer(segmentID)
s.wb.getOrCreateBuffer(segmentID, 0)
s.True(s.wb.HasSegment(segmentID))
}

View File

@ -6,6 +6,7 @@ const (
DirectoryWAL = "wal"
DirectorySegmentAssign = "segment-assign"
DirectoryVChannel = "vchannel"
DirectorySchema = "schema"
KeyConsumeCheckpoint = "consume-checkpoint"
)

View File

@ -2,8 +2,11 @@ package streamingnode
import (
"context"
"fmt"
"path"
"sort"
"strconv"
"strings"
"github.com/cockroachdb/errors"
"google.golang.org/protobuf/proto"
@ -14,6 +17,7 @@ import (
"github.com/milvus-io/milvus/pkg/v2/util"
"github.com/milvus-io/milvus/pkg/v2/util/etcd"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
// NewCataLog creates a new streaming-node catalog instance.
@ -26,7 +30,10 @@ import (
// │   ├── checkpoint
// │   ├── vchannels
// │   │   ├── vchannel-1
// │   │   └── vchannel-2
// │   │   │   ├── schema/version-1
// │   │   │   └── schema/version-2
// │   │   ├── vchannel-2
// │   │   │   └── schema/version-1
// │   └── segment-assign
// │   ├── 456398247934
// │   ├── 456398247936
@ -58,36 +65,72 @@ func (c *catalog) ListVChannel(ctx context.Context, pchannelName string) ([]*str
if err != nil {
return nil, err
}
infos := make([]*streamingpb.VChannelMeta, 0, len(values))
for k, value := range values {
info := &streamingpb.VChannelMeta{}
if err = proto.Unmarshal([]byte(value), info); err != nil {
return nil, errors.Wrapf(err, "unmarshal pchannel %s failed", keys[k])
return c.newVChannelMetaFromKV(prefix, keys, values)
}
// newVChannelMetaFromKV groups the vchannel meta by the vchannel name.
func (c *catalog) newVChannelMetaFromKV(prefix string, keys []string, values []string) ([]*streamingpb.VChannelMeta, error) {
keys = removePrefix(prefix, keys)
vchannels := make(map[string]*streamingpb.VChannelMeta, len(keys))
schemas := make(map[string][]*streamingpb.CollectionSchemaOfVChannel, len(keys))
for idx, key := range keys {
ks := strings.Split(key, "/")
switch len(ks) {
case 1:
// the vchannel vchannel path.
vchannel := &streamingpb.VChannelMeta{}
if err := proto.Unmarshal([]byte(values[idx]), vchannel); err != nil {
return nil, errors.Wrapf(err, "unmarshal vchannel meta %s failed", key)
}
vchannels[ks[0]] = vchannel
case 3: // {{vchannel}}/schema/{{version}}
// the schema path.
channelName := ks[0]
if ks[1] != DirectorySchema {
continue
}
schema := &streamingpb.CollectionSchemaOfVChannel{}
if err := proto.Unmarshal([]byte(values[idx]), schema); err != nil {
return nil, errors.Wrapf(err, "unmarshal schema %s failed", key)
}
if _, ok := schemas[channelName]; !ok {
schemas[channelName] = make([]*streamingpb.CollectionSchemaOfVChannel, 0, 2)
}
schemas[channelName] = append(schemas[channelName], schema)
}
infos = append(infos, info)
}
return infos, nil
vchannelsWithSchemas := make([]*streamingpb.VChannelMeta, 0, len(vchannels))
for vchannelName, vchannel := range vchannels {
schemas, ok := schemas[vchannelName]
if !ok {
panic(fmt.Sprintf("vchannel %s has no schemas in recovery info", vchannelName))
}
sort.Slice(schemas, func(i, j int) bool {
// order by checkpoint time tick.
return schemas[i].CheckpointTimeTick < schemas[j].CheckpointTimeTick
})
vchannel.CollectionInfo.Schemas = schemas
vchannelsWithSchemas = append(vchannelsWithSchemas, vchannel)
}
return vchannelsWithSchemas, nil
}
// SaveVChannels save vchannel on current pchannel.
func (c *catalog) SaveVChannels(ctx context.Context, pchannelName string, vchannels map[string]*streamingpb.VChannelMeta) error {
kvs := make(map[string]string, len(vchannels))
removes := make([]string, 0)
kvs := make(map[string]string, 2*len(vchannels))
removes := make([]string, 0, 2*len(vchannels))
for _, info := range vchannels {
key := buildVChannelMetaPathOfVChannel(pchannelName, info.GetVchannel())
if info.GetState() == streamingpb.VChannelState_VCHANNEL_STATE_DROPPED {
// Flushed segment should be removed from meta
removes = append(removes, key)
continue
}
data, err := proto.Marshal(info)
r, kv, err := c.getRemovalAndSaveForVChannel(pchannelName, info)
if err != nil {
return errors.Wrapf(err, "marshal vchannel %d at pchannel %s failed", info.GetVchannel(), pchannelName)
return err
}
removes = append(removes, r...)
for k, v := range kv {
kvs[k] = v
}
kvs[key] = string(data)
}
// TODO: We should perform a remove and save as a transaction but current the kv interface doesn't support it.
if len(removes) > 0 {
if err := etcd.RemoveByBatchWithLimit(removes, util.MaxEtcdTxnNum, func(partialRemoves []string) error {
return c.metaKV.MultiRemove(ctx, partialRemoves)
@ -103,6 +146,49 @@ func (c *catalog) SaveVChannels(ctx context.Context, pchannelName string, vchann
return nil
}
// getRemovalAndSaveForVChannel gets the removal and save for vchannel.
func (c *catalog) getRemovalAndSaveForVChannel(pchannelName string, info *streamingpb.VChannelMeta) ([]string, map[string]string, error) {
removes := make([]string, 0, len(info.CollectionInfo.Schemas)+1)
kvs := make(map[string]string, len(info.CollectionInfo.Schemas)+1)
key := buildVChannelMetaPathOfVChannel(pchannelName, info.GetVchannel())
if info.GetState() == streamingpb.VChannelState_VCHANNEL_STATE_DROPPED {
// Dropped vchannel should be removed from meta
for _, schema := range info.GetCollectionInfo().GetSchemas() {
// Also remove the schema of the vchannel.
removes = append(removes, buildVChannelSchemaPath(pchannelName, info.GetVchannel(), schema.GetCheckpointTimeTick()))
}
removes = append(removes, key)
return removes, kvs, nil
}
// Save the schema of the vchannel.
for _, schema := range info.GetCollectionInfo().GetSchemas() {
switch schema.State {
case streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_DROPPED:
// Dropped schema should be removed from meta
removes = append(removes, buildVChannelSchemaPath(pchannelName, info.GetVchannel(), schema.GetCheckpointTimeTick()))
default:
data, err := proto.Marshal(schema)
if err != nil {
return nil, nil, errors.Wrapf(err, "marshal schema %d at pchannel %s failed", schema.GetCheckpointTimeTick(), pchannelName)
}
kvs[buildVChannelSchemaPath(pchannelName, info.GetVchannel(), schema.GetCheckpointTimeTick())] = string(data)
}
}
// Schema is saved in the other key, so we don't need to save it in the vchannel meta.
// swap it first to marshal the vchannel meta without schema.
oldSchema := info.CollectionInfo.Schemas
info.CollectionInfo.Schemas = nil
data, err := proto.Marshal(info)
info.CollectionInfo.Schemas = oldSchema
if err != nil {
return nil, nil, errors.Wrapf(err, "marshal vchannel %d at pchannel %s failed", info.GetVchannel(), pchannelName)
}
kvs[key] = string(data)
return removes, kvs, nil
}
// ListSegmentAssignment lists the segment assignment info of the pchannel.
func (c *catalog) ListSegmentAssignment(ctx context.Context, pChannelName string) ([]*streamingpb.SegmentAssignmentMeta, error) {
prefix := buildSegmentAssignmentMetaPath(pChannelName)
@ -189,11 +275,24 @@ func buildVChannelMetaPath(pChannelName string) string {
return path.Join(buildWALDirectory(pChannelName), DirectoryVChannel) + "/"
}
// removePrefix removes the prefix from the keys.
func removePrefix(prefix string, keys []string) []string {
for idx, key := range keys {
keys[idx] = typeutil.After(key, prefix)
}
return keys
}
// buildVChannelMetaPathOfVChannel builds the path for vchannel meta
func buildVChannelMetaPathOfVChannel(pChannelName string, vchannelName string) string {
return path.Join(buildVChannelMetaPath(pChannelName), vchannelName)
}
// buildVChannelSchemaPath builds the path for vchannel schema
func buildVChannelSchemaPath(pChannelName string, vchannelName string, version uint64) string {
return path.Join(buildVChannelMetaPathOfVChannel(pChannelName, vchannelName), DirectorySchema, strconv.FormatUint(version, 10))
}
// buildSegmentAssignmentMetaPath builds the path for segment assignment
func buildSegmentAssignmentMetaPath(pChannelName string) string {
return path.Join(buildWALDirectory(pChannelName), DirectorySegmentAssign) + "/"

View File

@ -5,11 +5,15 @@ import (
"testing"
"github.com/cockroachdb/errors"
"github.com/google/uuid"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/kv/mocks"
kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv"
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
)
@ -80,33 +84,117 @@ func TestCatalogSegmentAssignments(t *testing.T) {
}
func TestCatalogVChannel(t *testing.T) {
kv := mocks.NewMetaKv(t)
k := "p1/vchannel-1"
v := streamingpb.VChannelMeta{}
vs, err := proto.Marshal(&v)
assert.NoError(t, err)
kv.EXPECT().LoadWithPrefix(mock.Anything, mock.Anything).Return([]string{k}, []string{string(vs)}, nil)
etcdCli, _ := kvfactory.GetEtcdAndPath()
rootPath := "testCatalogVChannel-" + uuid.New().String() + "/meta"
kv := etcdkv.NewEtcdKV(etcdCli, rootPath)
catalog := NewCataLog(kv)
ctx := context.Background()
metas, err := catalog.ListVChannel(ctx, "p1")
assert.Len(t, metas, 1)
channel1 := "p1"
vchannels, err := catalog.ListVChannel(ctx, channel1)
assert.Len(t, vchannels, 0)
assert.NoError(t, err)
kv.EXPECT().MultiRemove(mock.Anything, mock.Anything).Return(nil)
kv.EXPECT().MultiSave(mock.Anything, mock.Anything).Return(nil)
err = catalog.SaveVChannels(ctx, "p1", map[string]*streamingpb.VChannelMeta{
vchannelMetas := map[string]*streamingpb.VChannelMeta{
"vchannel-1": {
Vchannel: "vchannel-1",
State: streamingpb.VChannelState_VCHANNEL_STATE_DROPPED,
State: streamingpb.VChannelState_VCHANNEL_STATE_NORMAL,
CollectionInfo: &streamingpb.CollectionInfoOfVChannel{
CollectionId: 100,
Partitions: []*streamingpb.PartitionInfoOfVChannel{
{
PartitionId: 100,
},
},
Schemas: []*streamingpb.CollectionSchemaOfVChannel{
{
Schema: &schemapb.CollectionSchema{
Name: "collection-1",
},
CheckpointTimeTick: 0,
State: streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_DROPPED,
},
{
Schema: &schemapb.CollectionSchema{
Name: "collection-2",
},
CheckpointTimeTick: 8,
State: streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_NORMAL,
},
{
Schema: &schemapb.CollectionSchema{
Name: "collection-3",
},
CheckpointTimeTick: 101,
State: streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_NORMAL,
},
},
},
},
"vchannel-2": {
Vchannel: "vchannel-2",
State: streamingpb.VChannelState_VCHANNEL_STATE_NORMAL,
CollectionInfo: &streamingpb.CollectionInfoOfVChannel{
CollectionId: 100,
Partitions: []*streamingpb.PartitionInfoOfVChannel{
{
PartitionId: 100,
},
},
Schemas: []*streamingpb.CollectionSchemaOfVChannel{
{
Schema: &schemapb.CollectionSchema{
Name: "collection-1",
},
CheckpointTimeTick: 0,
State: streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_NORMAL,
},
},
},
},
})
}
err = catalog.SaveVChannels(ctx, channel1, vchannelMetas)
assert.NoError(t, err)
vchannels, err = catalog.ListVChannel(ctx, channel1)
assert.Len(t, vchannels, 2)
assert.NoError(t, err)
for _, vchannel := range vchannels {
switch vchannel.Vchannel {
case "vchannel-1":
assert.Len(t, vchannel.CollectionInfo.Schemas, 2)
assert.Equal(t, vchannel.CollectionInfo.Schemas[0].Schema.Name, "collection-2")
assert.Equal(t, vchannel.CollectionInfo.Schemas[0].CheckpointTimeTick, uint64(8))
assert.Equal(t, vchannel.CollectionInfo.Schemas[0].State, streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_NORMAL)
assert.Equal(t, vchannel.CollectionInfo.Schemas[1].Schema.Name, "collection-3")
assert.Equal(t, vchannel.CollectionInfo.Schemas[1].CheckpointTimeTick, uint64(101))
assert.Equal(t, vchannel.CollectionInfo.Schemas[1].State, streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_NORMAL)
case "vchannel-2":
assert.Len(t, vchannel.CollectionInfo.Schemas, 1)
assert.Equal(t, vchannel.CollectionInfo.Schemas[0].Schema.Name, "collection-1")
assert.Equal(t, vchannel.CollectionInfo.Schemas[0].CheckpointTimeTick, uint64(0))
assert.Equal(t, vchannel.CollectionInfo.Schemas[0].State, streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_NORMAL)
}
}
vchannelMetas["vchannel-1"].CollectionInfo.Schemas[1].State = streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_DROPPED
vchannelMetas["vchannel-2"].State = streamingpb.VChannelState_VCHANNEL_STATE_DROPPED
err = catalog.SaveVChannels(ctx, channel1, vchannelMetas)
assert.NoError(t, err)
vchannels, err = catalog.ListVChannel(ctx, channel1)
assert.Len(t, vchannels, 1)
assert.NoError(t, err)
for _, vchannel := range vchannels {
switch vchannel.Vchannel {
case "vchannel-1":
assert.Len(t, vchannel.CollectionInfo.Schemas, 1)
assert.Equal(t, vchannel.CollectionInfo.Schemas[0].Schema.Name, "collection-3")
assert.Equal(t, vchannel.CollectionInfo.Schemas[0].CheckpointTimeTick, uint64(101))
assert.Equal(t, vchannel.CollectionInfo.Schemas[0].State, streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_NORMAL)
}
}
}
func TestBuildDirectory(t *testing.T) {

View File

@ -9,6 +9,8 @@ import (
mock "github.com/stretchr/testify/mock"
recovery "github.com/milvus-io/milvus/internal/streamingnode/server/wal/recovery"
schemapb "github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
)
// MockRecoveryStorage is an autogenerated mock type for the RecoveryStorage type
@ -56,6 +58,66 @@ func (_c *MockRecoveryStorage_Close_Call) RunAndReturn(run func()) *MockRecovery
return _c
}
// GetSchema provides a mock function with given fields: ctx, vchannel, timetick
func (_m *MockRecoveryStorage) GetSchema(ctx context.Context, vchannel string, timetick uint64) (*schemapb.CollectionSchema, error) {
ret := _m.Called(ctx, vchannel, timetick)
if len(ret) == 0 {
panic("no return value specified for GetSchema")
}
var r0 *schemapb.CollectionSchema
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, string, uint64) (*schemapb.CollectionSchema, error)); ok {
return rf(ctx, vchannel, timetick)
}
if rf, ok := ret.Get(0).(func(context.Context, string, uint64) *schemapb.CollectionSchema); ok {
r0 = rf(ctx, vchannel, timetick)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*schemapb.CollectionSchema)
}
}
if rf, ok := ret.Get(1).(func(context.Context, string, uint64) error); ok {
r1 = rf(ctx, vchannel, timetick)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockRecoveryStorage_GetSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetSchema'
type MockRecoveryStorage_GetSchema_Call struct {
*mock.Call
}
// GetSchema is a helper method to define mock.On call
// - ctx context.Context
// - vchannel string
// - timetick uint64
func (_e *MockRecoveryStorage_Expecter) GetSchema(ctx interface{}, vchannel interface{}, timetick interface{}) *MockRecoveryStorage_GetSchema_Call {
return &MockRecoveryStorage_GetSchema_Call{Call: _e.mock.On("GetSchema", ctx, vchannel, timetick)}
}
func (_c *MockRecoveryStorage_GetSchema_Call) Run(run func(ctx context.Context, vchannel string, timetick uint64)) *MockRecoveryStorage_GetSchema_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(string), args[2].(uint64))
})
return _c
}
func (_c *MockRecoveryStorage_GetSchema_Call) Return(_a0 *schemapb.CollectionSchema, _a1 error) *MockRecoveryStorage_GetSchema_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockRecoveryStorage_GetSchema_Call) RunAndReturn(run func(context.Context, string, uint64) (*schemapb.CollectionSchema, error)) *MockRecoveryStorage_GetSchema_Call {
_c.Call.Return(run)
return _c
}
// Metrics provides a mock function with no fields
func (_m *MockRecoveryStorage) Metrics() recovery.RecoveryMetrics {
ret := _m.Called()
@ -148,9 +210,9 @@ func (_c *MockRecoveryStorage_ObserveMessage_Call) RunAndReturn(run func(context
return _c
}
// UpdateFlusherCheckpoint provides a mock function with given fields: checkpoint
func (_m *MockRecoveryStorage) UpdateFlusherCheckpoint(checkpoint *recovery.WALCheckpoint) {
_m.Called(checkpoint)
// UpdateFlusherCheckpoint provides a mock function with given fields: vchannel, checkpoint
func (_m *MockRecoveryStorage) UpdateFlusherCheckpoint(vchannel string, checkpoint *recovery.WALCheckpoint) {
_m.Called(vchannel, checkpoint)
}
// MockRecoveryStorage_UpdateFlusherCheckpoint_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateFlusherCheckpoint'
@ -159,14 +221,15 @@ type MockRecoveryStorage_UpdateFlusherCheckpoint_Call struct {
}
// UpdateFlusherCheckpoint is a helper method to define mock.On call
// - vchannel string
// - checkpoint *recovery.WALCheckpoint
func (_e *MockRecoveryStorage_Expecter) UpdateFlusherCheckpoint(checkpoint interface{}) *MockRecoveryStorage_UpdateFlusherCheckpoint_Call {
return &MockRecoveryStorage_UpdateFlusherCheckpoint_Call{Call: _e.mock.On("UpdateFlusherCheckpoint", checkpoint)}
func (_e *MockRecoveryStorage_Expecter) UpdateFlusherCheckpoint(vchannel interface{}, checkpoint interface{}) *MockRecoveryStorage_UpdateFlusherCheckpoint_Call {
return &MockRecoveryStorage_UpdateFlusherCheckpoint_Call{Call: _e.mock.On("UpdateFlusherCheckpoint", vchannel, checkpoint)}
}
func (_c *MockRecoveryStorage_UpdateFlusherCheckpoint_Call) Run(run func(checkpoint *recovery.WALCheckpoint)) *MockRecoveryStorage_UpdateFlusherCheckpoint_Call {
func (_c *MockRecoveryStorage_UpdateFlusherCheckpoint_Call) Run(run func(vchannel string, checkpoint *recovery.WALCheckpoint)) *MockRecoveryStorage_UpdateFlusherCheckpoint_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(*recovery.WALCheckpoint))
run(args[0].(string), args[1].(*recovery.WALCheckpoint))
})
return _c
}
@ -176,7 +239,7 @@ func (_c *MockRecoveryStorage_UpdateFlusherCheckpoint_Call) Return() *MockRecove
return _c
}
func (_c *MockRecoveryStorage_UpdateFlusherCheckpoint_Call) RunAndReturn(run func(*recovery.WALCheckpoint)) *MockRecoveryStorage_UpdateFlusherCheckpoint_Call {
func (_c *MockRecoveryStorage_UpdateFlusherCheckpoint_Call) RunAndReturn(run func(string, *recovery.WALCheckpoint)) *MockRecoveryStorage_UpdateFlusherCheckpoint_Call {
_c.Run(run)
return _c
}

View File

@ -362,6 +362,10 @@ func (m *ChannelDistManager) GetShardLeader(channelName string, replica *Replica
// When upgrading from 2.5 to 2.6, the delegator leader may not locate at streaming node.
// We always use the streaming node as the delegator leader to avoid the delete data lost when loading segment.
candidates = channel
} else if !channelIsStreamingNode && candidateIsStreamingNode {
// When downgrading from 2.6 to 2.5, the delegator leader may locate at non-streaming node.
// We always use the non-streaming node as the delegator leader to avoid the delete data lost when loading segment.
continue
} else {
updateNeeded := false
switch {

View File

@ -368,7 +368,10 @@ func (suite *ChannelDistManagerSuite) TestGetShardLeader() {
})
defer snmanager.ResetStreamingNodeManager()
snmanager.StaticStreamingNodeManager.SetBalancerReady(balancer)
time.Sleep(50 * time.Millisecond)
suite.Eventually(func() bool {
nodeIDs := snmanager.StaticStreamingNodeManager.GetStreamingQueryNodeIDs()
return nodeIDs.Contain(4)
}, 10*time.Second, 100*time.Millisecond)
channel1Node4 := suite.channels["dmc0"].Clone()
channel1Node4.Node = 4

View File

@ -507,8 +507,7 @@ func (s *WriteSchemaChangeWALStep) Execute(ctx context.Context) ([]nestedStep, e
CollectionId: s.collection.CollectionID,
}).
WithBody(&message.SchemaChangeMessageBody{
Schema: schema,
ModifyTs: s.ts,
Schema: schema,
}).BuildBroadcast()
if err != nil {
return nil, err

View File

@ -3,6 +3,8 @@ package flusherimpl
import (
"context"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/flushcommon/pipeline"
"github.com/milvus-io/milvus/internal/streamingnode/server/resource"
"github.com/milvus-io/milvus/pkg/v2/mq/msgstream"
@ -15,22 +17,25 @@ func newDataSyncServiceWrapper(
channelName string,
input chan<- *msgstream.MsgPack,
ds *pipeline.DataSyncService,
channelCheckpointTimeTick uint64,
) *dataSyncServiceWrapper {
handler := adaptor.NewBaseMsgPackAdaptorHandler()
return &dataSyncServiceWrapper{
channelName: channelName,
input: input,
handler: handler,
ds: ds,
channelName: channelName,
input: input,
handler: handler,
ds: ds,
channelCheckpointTimeTick: channelCheckpointTimeTick,
}
}
// dataSyncServiceWrapper wraps DataSyncService and related input channel.
type dataSyncServiceWrapper struct {
channelName string
input chan<- *msgstream.MsgPack
handler *adaptor.BaseMsgPackAdaptorHandler
ds *pipeline.DataSyncService
channelName string
channelCheckpointTimeTick uint64
input chan<- *msgstream.MsgPack
handler *adaptor.BaseMsgPackAdaptorHandler
ds *pipeline.DataSyncService
}
// Start starts the data sync service.
@ -44,6 +49,16 @@ func (ds *dataSyncServiceWrapper) HandleMessage(ctx context.Context, msg message
for ds.handler.PendingMsgPack.Len() > 0 {
next := ds.handler.PendingMsgPack.Next()
nextTsMsg := msgstream.MustBuildMsgPackFromConsumeMsgPack(next, adaptor.UnmashalerDispatcher)
// filter out the message less than vchannel level checkpoint.
if nextTsMsg.EndTs < ds.channelCheckpointTimeTick {
ds.handler.Logger.Debug("skip the message less than vchannel checkpoint",
zap.Uint64("timestamp", nextTsMsg.EndTs),
zap.Uint64("checkpoint", ds.channelCheckpointTimeTick),
)
ds.handler.PendingMsgPack.UnsafeAdvance()
continue
}
select {
case <-ctx.Done():
return ctx.Err()

View File

@ -16,6 +16,7 @@ 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/shard/stats"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/recovery"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/utility"
"github.com/milvus-io/milvus/internal/util/idalloc"
"github.com/milvus-io/milvus/pkg/v2/log"
@ -36,10 +37,13 @@ type flusherComponents struct {
dataServices map[string]*dataSyncServiceWrapper
logger *log.MLogger
recoveryCheckPointTimeTick uint64 // The time tick of the recovery storage.
rs recovery.RecoveryStorage
}
// WhenCreateCollection handles the create collection message.
func (impl *flusherComponents) WhenCreateCollection(createCollectionMsg message.ImmutableCreateCollectionMessageV1) {
// because we need to get the schema from the recovery storage, we need to observe the message at recovery storage first.
impl.rs.ObserveMessage(context.Background(), createCollectionMsg)
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.
@ -76,6 +80,7 @@ func (impl *flusherComponents) WhenCreateCollection(createCollectionMsg message.
CheckpointUpdater: impl.cpUpdater,
Allocator: idalloc.NewMAllocator(resource.Resource().IDAllocator()),
MsgHandler: newMsgHandler(resource.Resource().WriteBufferManager()),
SchemaManager: newVersionedSchemaManager(createCollectionMsg.VChannel(), impl.rs),
},
msgChan,
&datapb.VchannelInfo{
@ -89,7 +94,6 @@ func (impl *flusherComponents) WhenCreateCollection(createCollectionMsg message.
Timestamp: createCollectionMsg.TimeTick(),
},
},
schema,
func(t syncmgr.Task, err error) {
if err != nil || t == nil {
return
@ -145,7 +149,7 @@ func (impl *flusherComponents) addNewDataSyncService(
input chan<- *msgstream.MsgPack,
ds *pipeline.DataSyncService,
) {
newDS := newDataSyncServiceWrapper(createCollectionMsg.VChannel(), input, ds)
newDS := newDataSyncServiceWrapper(createCollectionMsg.VChannel(), input, ds, createCollectionMsg.TimeTick())
newDS.Start()
impl.dataServices[createCollectionMsg.VChannel()] = newDS
impl.logger.Info("create data sync service done", zap.String("vchannel", createCollectionMsg.VChannel()))
@ -244,6 +248,8 @@ func (impl *flusherComponents) buildDataSyncServiceWithRetry(ctx context.Context
func (impl *flusherComponents) buildDataSyncService(ctx context.Context, recoverInfo *datapb.GetChannelRecoveryInfoResponse) (*dataSyncServiceWrapper, error) {
// Build and add pipeline.
input := make(chan *msgstream.MsgPack, 10)
schemaManager := newVersionedSchemaManager(recoverInfo.GetInfo().GetChannelName(), impl.rs)
schema := schemaManager.GetSchema(0)
ds, err := pipeline.NewStreamingNodeDataSyncService(ctx,
&util.PipelineParams{
Ctx: context.Background(),
@ -254,8 +260,9 @@ func (impl *flusherComponents) buildDataSyncService(ctx context.Context, recover
CheckpointUpdater: impl.cpUpdater,
Allocator: idalloc.NewMAllocator(resource.Resource().IDAllocator()),
MsgHandler: newMsgHandler(resource.Resource().WriteBufferManager()),
SchemaManager: newVersionedSchemaManager(recoverInfo.GetInfo().GetChannelName(), impl.rs),
},
&datapb.ChannelWatchInfo{Vchan: recoverInfo.GetInfo(), Schema: recoverInfo.GetSchema()},
&datapb.ChannelWatchInfo{Vchan: recoverInfo.GetInfo(), Schema: schema},
input,
func(t syncmgr.Task, err error) {
if err != nil || t == nil {
@ -274,5 +281,5 @@ func (impl *flusherComponents) buildDataSyncService(ctx context.Context, recover
if err != nil {
return nil, err
}
return newDataSyncServiceWrapper(recoverInfo.Info.ChannelName, input, ds), nil
return newDataSyncServiceWrapper(recoverInfo.Info.ChannelName, input, ds, recoverInfo.Info.GetSeekPosition().GetTimestamp()), nil
}

View File

@ -0,0 +1,31 @@
package flusherimpl
import (
"context"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/recovery"
)
// newVersionedSchemaManager creates a new versioned schema manager.
func newVersionedSchemaManager(vchannel string, rs recovery.RecoveryStorage) *versionedSchemaManager {
return &versionedSchemaManager{
vchannel: vchannel,
rs: rs,
}
}
// versionedSchemaManager is a schema manager that gets the schema from the recovery storage.
// It is used to get the schema of the vchannel at the given timetick.
type versionedSchemaManager struct {
vchannel string
rs recovery.RecoveryStorage
}
func (m *versionedSchemaManager) GetSchema(timetick uint64) *schemapb.CollectionSchema {
schema, err := m.rs.GetSchema(context.Background(), m.vchannel, timetick)
if err != nil {
panic(err)
}
return schema
}

View File

@ -152,7 +152,7 @@ func (impl *WALFlusherImpl) buildFlusherComponents(ctx context.Context, l wal.WA
cpUpdater := util.NewChannelCheckpointUpdaterWithCallback(broker, func(mp *msgpb.MsgPosition) {
messageID := adaptor.MustGetMessageIDFromMQWrapperIDBytes(l.WALName(), mp.MsgID)
impl.RecoveryStorage.UpdateFlusherCheckpoint(&recovery.WALCheckpoint{
impl.RecoveryStorage.UpdateFlusherCheckpoint(mp.ChannelName, &recovery.WALCheckpoint{
MessageID: messageID,
TimeTick: mp.Timestamp,
Magic: recovery.RecoveryMagicStreamingInitialized,
@ -168,6 +168,7 @@ func (impl *WALFlusherImpl) buildFlusherComponents(ctx context.Context, l wal.WA
dataServices: make(map[string]*dataSyncServiceWrapper),
logger: impl.logger,
recoveryCheckPointTimeTick: snapshot.Checkpoint.TimeTick,
rs: impl.RecoveryStorage,
}
impl.logger.Info("flusher components intiailizing done")
if err := fc.recover(ctx, recoverInfos); err != nil {

View File

@ -55,6 +55,12 @@ func TestWALFlusher(t *testing.T) {
fMixcoord := syncutil.NewFuture[internaltypes.MixCoordClient]()
fMixcoord.Set(mixcoord)
rs := mock_recovery.NewMockRecoveryStorage(t)
rs.EXPECT().GetSchema(mock.Anything, mock.Anything, mock.Anything).Return(&schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{FieldID: 100, Name: "ID", IsPrimaryKey: true, DataType: schemapb.DataType_Int64},
{FieldID: 101, Name: "Vector", DataType: schemapb.DataType_FloatVector},
},
}, nil)
rs.EXPECT().ObserveMessage(mock.Anything, mock.Anything).Return(nil)
rs.EXPECT().Close().Return()
resource.InitForTest(

View File

@ -5,7 +5,6 @@ import (
"time"
"github.com/cenkalti/backoff/v4"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"go.uber.org/zap"
"go.uber.org/zap/zapcore"
@ -206,9 +205,12 @@ func (rs *recoveryStorageImpl) retryOperationWithBackoff(ctx context.Context, lo
if err == nil {
return nil
}
if errors.IsAny(err, context.Canceled, context.DeadlineExceeded) {
return err
// because underlying kv may report the context.Canceled, context.DeadlineExceeded even if the ctx is not canceled.
// so we cannot use errors.IsAny(err, context.Canceled, context.DeadlineExceeded) to check the error.
if ctx.Err() != nil {
return ctx.Err()
}
nextInterval := backoff.NextBackOff()
logger.Warn("failed to persist operation, wait for retry...", zap.Duration("nextRetryInterval", nextInterval), zap.Error(err))
select {

View File

@ -2,10 +2,13 @@ package recovery
import (
"context"
"fmt"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/streamingnode/server/resource"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
@ -95,6 +98,10 @@ func (r *recoveryStorageImpl) initializeRecoverInfo(ctx context.Context, channel
if err = merr.CheckRPCCall(resp, err); err != nil {
return nil, errors.Wrap(err, "failed to get pchannel info from rootcoord")
}
schemas, err := r.fetchLatestSchemaFromCoord(ctx, resp)
if err != nil {
return nil, errors.Wrap(err, "failed to fetch latest schema from coord")
}
// save the vchannel recovery info into the catalog
vchannels := make(map[string]*streamingpb.VChannelMeta, len(resp.GetCollections()))
@ -120,13 +127,26 @@ func (r *recoveryStorageImpl) initializeRecoverInfo(ctx context.Context, channel
for _, partition := range collection.Partitions {
partitions = append(partitions, &streamingpb.PartitionInfoOfVChannel{PartitionId: partition.PartitionId})
}
if schemas[collection.CollectionId] == nil {
panic(fmt.Sprintf("schema not found for collection, %d", collection.CollectionId))
}
vchannels[collection.Vchannel] = &streamingpb.VChannelMeta{
Vchannel: collection.Vchannel,
State: streamingpb.VChannelState_VCHANNEL_STATE_NORMAL,
CollectionInfo: &streamingpb.CollectionInfoOfVChannel{
CollectionId: collection.CollectionId,
Partitions: partitions,
Schemas: []*streamingpb.CollectionSchemaOfVChannel{
{
Schema: schemas[collection.CollectionId].Schema,
State: streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_NORMAL,
CheckpointTimeTick: 0, // The recovery info from old arch should be set as zero.
// because we don't have the version before streaming service is enabled.
// all message will happen after the recovery info is initialized.
},
},
},
CheckpointTimeTick: 0, // same as schema above.
}
}
@ -153,3 +173,45 @@ func (r *recoveryStorageImpl) initializeRecoverInfo(ctx context.Context, channel
)
return checkpoint, nil
}
// fetchLatestSchemaFromCoord fetches the latest schema from coord.
func (r *recoveryStorageImpl) fetchLatestSchemaFromCoord(ctx context.Context, resp *rootcoordpb.GetPChannelInfoResponse) (map[int64]*streamingpb.CollectionSchemaOfVChannel, error) {
rc, err := resource.Resource().MixCoordClient().GetWithContext(ctx)
if err != nil {
return nil, errors.Wrap(err, "failed to get coord client")
}
futures := make([]*conc.Future[*milvuspb.DescribeCollectionResponse], 0, len(resp.GetCollections()))
for _, collection := range resp.GetCollections() {
if collection.State == etcdpb.CollectionState_CollectionDropping {
continue
}
future := conc.Go(func() (*milvuspb.DescribeCollectionResponse, error) {
resp, err := rc.DescribeCollectionInternal(ctx, &milvuspb.DescribeCollectionRequest{
Base: commonpbutil.NewMsgBase(
commonpbutil.WithMsgType(commonpb.MsgType_DescribeCollection),
commonpbutil.WithSourceID(paramtable.GetNodeID()),
),
CollectionID: collection.CollectionId,
})
if err = merr.CheckRPCCall(resp, err); err != nil {
return nil, errors.Wrap(err, "failed to describe collection")
}
return resp, nil
})
futures = append(futures, future)
}
if err := conc.BlockOnAll(futures...); err != nil {
return nil, errors.Wrap(err, "failed to describe collection")
}
schemas := make(map[int64]*streamingpb.CollectionSchemaOfVChannel, len(futures))
for _, future := range futures {
resp := future.Value()
collectionID := resp.CollectionID
schemas[collectionID] = &streamingpb.CollectionSchemaOfVChannel{
Schema: resp.Schema,
}
}
return schemas, nil
}

View File

@ -11,6 +11,8 @@ import (
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"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/streamingnode/server/resource"
@ -118,6 +120,13 @@ func TestInitRecoveryInfoFromCoord(t *testing.T) {
},
},
}, nil)
c.EXPECT().DescribeCollectionInternal(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, req *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error) {
return &milvuspb.DescribeCollectionResponse{
Status: merr.Success(),
CollectionID: req.CollectionID,
Schema: &schemapb.CollectionSchema{},
}, nil
})
c.EXPECT().DropVirtualChannel(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, req *datapb.DropVirtualChannelRequest, opts ...grpc.CallOption) (*datapb.DropVirtualChannelResponse, error) {
assert.Equal(t, "v3", req.GetChannelName())
return &datapb.DropVirtualChannelResponse{

View File

@ -3,6 +3,7 @@ package recovery
import (
"context"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/streamingnode/server/wal/utility"
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
@ -68,12 +69,16 @@ type RecoveryStorage interface {
// Metrics gets the metrics of the recovery storage.
Metrics() RecoveryMetrics
// TODO: should be removed in future,
// GetSchema gets last schema of the collection which timetick is less than the given timetick.
GetSchema(ctx context.Context, vchannel string, timetick uint64) (*schemapb.CollectionSchema, error)
// ObserveMessage observes the message from the WAL.
ObserveMessage(ctx context.Context, msg message.ImmutableMessage) error
// UpdateFlusherCheckpoint updates the checkpoint of flusher.
// TODO: should be removed in future, after merge the flusher logic into recovery storage.
UpdateFlusherCheckpoint(checkpoint *WALCheckpoint)
UpdateFlusherCheckpoint(vchannel string, checkpoint *WALCheckpoint)
// Close closes the recovery storage.
Close()

View File

@ -4,9 +4,11 @@ import (
"context"
"sync"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/distributed/streaming"
"github.com/milvus-io/milvus/internal/streamingnode/server/resource"
"github.com/milvus-io/milvus/pkg/v2/log"
@ -85,7 +87,6 @@ type recoveryStorageImpl struct {
segments map[int64]*segmentRecoveryInfo
vchannels map[string]*vchannelRecoveryInfo
checkpoint *WALCheckpoint
flusherCheckpoint *WALCheckpoint
dirtyCounter int // records the message count since last persist snapshot.
// used to trigger the recovery persist operation.
persistNotifier chan struct{}
@ -107,15 +108,33 @@ func (r *recoveryStorageImpl) Metrics() RecoveryMetrics {
// UpdateFlusherCheckpoint updates the checkpoint of flusher.
// TODO: should be removed in future, after merge the flusher logic into recovery storage.
func (r *recoveryStorageImpl) UpdateFlusherCheckpoint(checkpoint *WALCheckpoint) {
func (r *recoveryStorageImpl) UpdateFlusherCheckpoint(vchannel string, checkpoint *WALCheckpoint) {
r.mu.Lock()
defer r.mu.Unlock()
if r.flusherCheckpoint == nil || r.flusherCheckpoint.MessageID.LTE(checkpoint.MessageID) {
r.flusherCheckpoint = checkpoint
r.Logger().Info("update checkpoint of flusher", zap.String("messageID", checkpoint.MessageID.String()), zap.Uint64("timeTick", checkpoint.TimeTick))
if vchannelInfo, ok := r.vchannels[vchannel]; ok {
if err := vchannelInfo.UpdateFlushCheckpoint(checkpoint); err != nil {
r.Logger().Warn("failed to update flush checkpoint", zap.Error(err))
return
}
r.Logger().Info("update flush checkpoint", zap.String("vchannel", vchannel), zap.String("messageID", checkpoint.MessageID.String()), zap.Uint64("timeTick", checkpoint.TimeTick))
return
}
r.Logger().Warn("update illegal checkpoint of flusher", zap.String("current", r.flusherCheckpoint.MessageID.String()), zap.String("target", checkpoint.MessageID.String()))
r.Logger().Warn("vchannel not found", zap.String("vchannel", vchannel))
}
// GetSchema gets the schema of the collection at the given timetick.
func (r *recoveryStorageImpl) GetSchema(ctx context.Context, vchannel string, timetick uint64) (*schemapb.CollectionSchema, error) {
r.mu.Lock()
defer r.mu.Unlock()
if vchannelInfo, ok := r.vchannels[vchannel]; ok {
_, schema := vchannelInfo.GetSchema(timetick)
if schema == nil {
return nil, errors.Errorf("critical error: schema not found, vchannel: %s, timetick: %d", vchannel, timetick)
}
return schema, nil
}
return nil, errors.Errorf("critical error: vchannel not found, vchannel: %s, timetick: %d", vchannel, timetick)
}
// ObserveMessage is called when a new message is observed.
@ -209,6 +228,10 @@ func (r *recoveryStorageImpl) observeMessage(msg message.ImmutableMessage) {
r.checkpoint.MessageID = msg.LastConfirmedMessageID()
r.metrics.ObServeInMemMetrics(r.checkpoint.TimeTick)
if !msg.IsPersisted() {
// only trigger persist when the message is persisted.
return
}
r.dirtyCounter++
if r.dirtyCounter > r.cfg.maxDirtyMessages {
r.notifyPersist()
@ -384,12 +407,12 @@ func (r *recoveryStorageImpl) handleDropPartition(msg message.ImmutableDropParti
}
r.vchannels[msg.VChannel()].ObserveDropPartition(msg)
// flush all existing segments.
r.flushAllSegmentOfPartition(msg, msg.Header().CollectionId, msg.Header().PartitionId)
r.flushAllSegmentOfPartition(msg, msg.Header().PartitionId)
r.Logger().Info("drop partition", log.FieldMessage(msg))
}
// flushAllSegmentOfPartition flushes all segments of the partition.
func (r *recoveryStorageImpl) flushAllSegmentOfPartition(msg message.ImmutableMessage, collectionID int64, partitionID int64) {
func (r *recoveryStorageImpl) flushAllSegmentOfPartition(msg message.ImmutableMessage, partitionID int64) {
segmentIDs := make([]int64, 0)
rows := make([]uint64, 0)
for _, segment := range r.segments {
@ -422,7 +445,11 @@ func (r *recoveryStorageImpl) handleSchemaChange(msg message.ImmutableSchemaChan
segments[segmentID] = struct{}{}
}
r.flushSegments(msg, segments)
// TODO: persist the schema change into recoveryinfo
// persist the schema change into recovery info.
if vchannelInfo, ok := r.vchannels[msg.VChannel()]; ok {
vchannelInfo.ObserveSchemaChange(msg)
}
}
// detectInconsistency detects the inconsistency in the recovery storage.
@ -441,5 +468,16 @@ func (r *recoveryStorageImpl) detectInconsistency(msg message.ImmutableMessage,
func (r *recoveryStorageImpl) getFlusherCheckpoint() *WALCheckpoint {
r.mu.Lock()
defer r.mu.Unlock()
return r.flusherCheckpoint
var minimumCheckpoint *WALCheckpoint
for _, vchannel := range r.vchannels {
if vchannel.GetFlushCheckpoint() == nil {
// If any flush checkpoint is not set, not ready.
return nil
}
if minimumCheckpoint == nil || vchannel.GetFlushCheckpoint().MessageID.LTE(minimumCheckpoint.MessageID) {
minimumCheckpoint = vchannel.GetFlushCheckpoint()
}
}
return minimumCheckpoint
}

View File

@ -1,8 +1,12 @@
package recovery
import (
"math"
"github.com/cockroachdb/errors"
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/pkg/v2/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
)
@ -27,6 +31,10 @@ func newVChannelRecoveryInfoFromCreateCollectionMessage(msg message.ImmutableCre
PartitionId: partitionId,
})
}
schema := &schemapb.CollectionSchema{}
if err := proto.Unmarshal(msg.MustBody().Schema, schema); err != nil {
panic("failed to unmarshal collection schema, err: " + err.Error())
}
return &vchannelRecoveryInfo{
meta: &streamingpb.VChannelMeta{
Vchannel: msg.VChannel(),
@ -34,6 +42,13 @@ func newVChannelRecoveryInfoFromCreateCollectionMessage(msg message.ImmutableCre
CollectionInfo: &streamingpb.CollectionInfoOfVChannel{
CollectionId: msg.Header().CollectionId,
Partitions: partitions,
Schemas: []*streamingpb.CollectionSchemaOfVChannel{
{
Schema: schema,
State: streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_NORMAL,
CheckpointTimeTick: msg.TimeTick(),
},
},
},
CheckpointTimeTick: msg.TimeTick(),
},
@ -44,8 +59,9 @@ func newVChannelRecoveryInfoFromCreateCollectionMessage(msg message.ImmutableCre
// vchannelRecoveryInfo is the recovery info for a vchannel.
type vchannelRecoveryInfo struct {
meta *streamingpb.VChannelMeta
dirty bool // whether the vchannel recovery info is dirty.
meta *streamingpb.VChannelMeta
flusherCheckpoint *WALCheckpoint // update from the flusher.
dirty bool // whether the vchannel recovery info is dirty.
}
// IsActive returns true if the vchannel is active.
@ -63,6 +79,67 @@ func (info *vchannelRecoveryInfo) IsPartitionActive(partitionId int64) bool {
return false
}
// GetFlushCheckpoint returns the flush checkpoint of the vchannel recovery info.
// return nil if the flush checkpoint is not set.
func (info *vchannelRecoveryInfo) GetFlushCheckpoint() *WALCheckpoint {
return info.flusherCheckpoint
}
// GetSchema returns the schema of the vchannel at the given timetick.
// return nil if the schema is not found.
func (info *vchannelRecoveryInfo) GetSchema(timetick uint64) (int, *schemapb.CollectionSchema) {
if timetick == 0 {
// timetick 0 means the latest schema.
timetick = math.MaxUint64
}
for i := len(info.meta.CollectionInfo.Schemas) - 1; i >= 0; i-- {
schema := info.meta.CollectionInfo.Schemas[i]
if schema.CheckpointTimeTick <= timetick {
return i, schema.Schema
}
}
return -1, nil
}
// UpdateFlushCheckpoint updates the flush checkpoint of the vchannel recovery info.
func (info *vchannelRecoveryInfo) UpdateFlushCheckpoint(checkpoint *WALCheckpoint) error {
if info.flusherCheckpoint == nil || info.flusherCheckpoint.MessageID.LTE(checkpoint.MessageID) {
info.flusherCheckpoint = checkpoint
idx, _ := info.GetSchema(info.flusherCheckpoint.TimeTick)
for i := 0; i < idx; i++ {
// drop the schema that is not used anymore.
// the future GetSchema operation will use the timetick greater than the flusher checkpoint.
// Those schema is too old, and will not be used anymore, can be dropped.
if info.meta.CollectionInfo.Schemas[i].State == streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_NORMAL {
info.meta.CollectionInfo.Schemas[i].State = streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_DROPPED
info.dirty = true
}
}
return nil
}
return errors.Errorf("update illegal checkpoint of flusher, current: %s, target: %s", info.flusherCheckpoint.MessageID.String(), checkpoint.MessageID.String())
}
// ObserveSchemaChange is called when a schema change message is observed.
func (info *vchannelRecoveryInfo) ObserveSchemaChange(msg message.ImmutableSchemaChangeMessageV2) {
if msg.TimeTick() < info.meta.CheckpointTimeTick {
// the txn message will share the same time tick.
// (although the flush operation is not a txn message)
// so we only filter the time tick is less than the checkpoint time tick.
// Consistent state is guaranteed by the recovery storage's mutex.
return
}
info.meta.CollectionInfo.Schemas = append(info.meta.CollectionInfo.Schemas, &streamingpb.CollectionSchemaOfVChannel{
Schema: msg.MustBody().Schema,
State: streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_NORMAL,
CheckpointTimeTick: msg.TimeTick(),
})
info.meta.CheckpointTimeTick = msg.TimeTick()
info.dirty = true
}
// ObserveDropCollection is called when a drop collection message is observed.
func (info *vchannelRecoveryInfo) ObserveDropCollection(msg message.ImmutableDropCollectionMessageV1) {
if msg.TimeTick() < info.meta.CheckpointTimeTick {
@ -129,6 +206,20 @@ func (info *vchannelRecoveryInfo) ConsumeDirtyAndGetSnapshot() (dirtySnapshot *s
if !info.dirty {
return nil, info.meta.State == streamingpb.VChannelState_VCHANNEL_STATE_DROPPED
}
// create the snapshot of the vchannel recovery info first.
snapshot := proto.Clone(info.meta).(*streamingpb.VChannelMeta)
// consume the dirty part of the vchannel recovery info.
for i := len(info.meta.CollectionInfo.Schemas) - 1; i >= 0; i-- {
// the schema is always dropped by timetick order,
// so we find the max index of the schema that is dropped,
// and drop all schema before it.
// the last schema is always normal, so it's safe to drop the schema by range.
if info.meta.CollectionInfo.Schemas[i].State == streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_DROPPED {
info.meta.CollectionInfo.Schemas = info.meta.CollectionInfo.Schemas[i+1:]
break
}
}
info.dirty = false
return proto.Clone(info.meta).(*streamingpb.VChannelMeta), info.meta.State == streamingpb.VChannelState_VCHANNEL_STATE_DROPPED
return snapshot, info.meta.State == streamingpb.VChannelState_VCHANNEL_STATE_DROPPED
}

View File

@ -4,8 +4,10 @@ import (
"testing"
"github.com/stretchr/testify/assert"
"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/pkg/v2/proto/streamingpb"
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
"github.com/milvus-io/milvus/pkg/v2/streaming/walimpls/impls/rmq"
@ -31,6 +33,11 @@ func TestNewVChannelRecoveryInfoFromVChannelMeta(t *testing.T) {
}
func TestNewVChannelRecoveryInfoFromCreateCollectionMessage(t *testing.T) {
schema1 := &schemapb.CollectionSchema{
Name: "test-collection-1",
}
schema1Bytes, _ := proto.Marshal(schema1)
// CreateCollection
msg := message.NewCreateCollectionMessageBuilderV1().
WithHeader(&message.CreateCollectionMessageHeader{
@ -41,6 +48,7 @@ func TestNewVChannelRecoveryInfoFromCreateCollectionMessage(t *testing.T) {
CollectionName: "test-collection",
CollectionID: 100,
PartitionIDs: []int64{101, 102},
Schema: schema1Bytes,
}).
WithVChannel("vchannel-1").
MustBuildMutable()
@ -53,12 +61,16 @@ func TestNewVChannelRecoveryInfoFromCreateCollectionMessage(t *testing.T) {
assert.Equal(t, streamingpb.VChannelState_VCHANNEL_STATE_NORMAL, info.meta.State)
assert.Equal(t, ts, info.meta.CheckpointTimeTick)
assert.Len(t, info.meta.CollectionInfo.Partitions, 2)
idx, schema1Saved := info.GetSchema(0)
assert.Equal(t, 0, idx)
assert.True(t, proto.Equal(schema1, schema1Saved))
assert.True(t, info.dirty)
snapshot, shouldBeRemoved := info.ConsumeDirtyAndGetSnapshot()
assert.NotNil(t, snapshot)
assert.False(t, shouldBeRemoved)
assert.False(t, info.dirty)
assert.Equal(t, 1, len(info.meta.CollectionInfo.Schemas))
snapshot, shouldBeRemoved = info.ConsumeDirtyAndGetSnapshot()
assert.Nil(t, snapshot)
@ -167,6 +179,75 @@ func TestNewVChannelRecoveryInfoFromCreateCollectionMessage(t *testing.T) {
assert.False(t, shouldBeRemoved)
assert.False(t, info.dirty)
// SchemaChange
schema2 := &schemapb.CollectionSchema{
Name: "test-collection-2",
}
msg5 := message.NewSchemaChangeMessageBuilderV2().
WithHeader(&message.SchemaChangeMessageHeader{
CollectionId: 100,
}).
WithBody(&message.SchemaChangeMessageBody{
Schema: schema2,
}).
WithVChannel("vchannel-1").
MustBuildMutable()
msgID5 := rmq.NewRmqID(5)
ts += 1
immutableMsg5 := msg5.WithTimeTick(ts).WithLastConfirmed(msgID5).IntoImmutableMessage(msgID5)
info.ObserveSchemaChange(message.MustAsImmutableCollectionSchemaChangeV2(immutableMsg5))
idx, schema2Saved := info.GetSchema(0)
assert.Equal(t, 1, idx)
assert.True(t, proto.Equal(schema2, schema2Saved))
idx, schema2Saved = info.GetSchema(ts)
assert.Equal(t, 1, idx)
assert.True(t, proto.Equal(schema2, schema2Saved))
idx, schema2Saved = info.GetSchema(ts - 1)
assert.Equal(t, 0, idx)
assert.True(t, proto.Equal(schema1, schema2Saved))
assert.True(t, info.dirty)
snapshot, shouldBeRemoved = info.ConsumeDirtyAndGetSnapshot()
assert.NotNil(t, snapshot)
assert.False(t, shouldBeRemoved)
assert.False(t, info.dirty)
assert.Len(t, snapshot.CollectionInfo.Schemas, 2)
// UpdateFlushCheckpoint
info.UpdateFlushCheckpoint(&WALCheckpoint{
MessageID: msgID5,
TimeTick: ts - 1,
})
assert.Equal(t, ts-1, info.flusherCheckpoint.TimeTick)
assert.False(t, info.dirty)
// schema change will be dropped by the flusher checkpoint.
info.UpdateFlushCheckpoint(&WALCheckpoint{
MessageID: msgID5,
TimeTick: ts + 1,
})
assert.Equal(t, ts+1, info.flusherCheckpoint.TimeTick)
assert.True(t, info.dirty)
idx, schema2Saved = info.GetSchema(ts - 1)
assert.Equal(t, 0, idx)
assert.Equal(t, streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_DROPPED, info.meta.CollectionInfo.Schemas[idx].State)
assert.Equal(t, schema1.Name, schema2Saved.Name)
idx, schema2Saved = info.GetSchema(ts)
assert.Equal(t, 1, idx)
assert.Equal(t, streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_NORMAL, info.meta.CollectionInfo.Schemas[idx].State)
assert.Equal(t, schema2.Name, schema2Saved.Name)
snapshot, shouldBeRemoved = info.ConsumeDirtyAndGetSnapshot()
assert.NotNil(t, snapshot)
assert.False(t, shouldBeRemoved)
assert.False(t, info.dirty)
assert.Len(t, info.meta.CollectionInfo.Schemas, 1)
assert.True(t, proto.Equal(schema2, info.meta.CollectionInfo.Schemas[0].Schema))
assert.Len(t, snapshot.CollectionInfo.Schemas, 2)
assert.Equal(t, streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_DROPPED, snapshot.CollectionInfo.Schemas[0].State)
assert.Equal(t, streamingpb.VChannelSchemaState_VCHANNEL_SCHEMA_STATE_NORMAL, snapshot.CollectionInfo.Schemas[1].State)
// DropCollection
msg2 := message.NewDropCollectionMessageBuilderV1().
WithHeader(&message.DropCollectionMessageHeader{

View File

@ -516,7 +516,7 @@ message GetChannelRecoveryInfoRequest {
message GetChannelRecoveryInfoResponse {
common.Status status = 1;
VchannelInfo info = 2;
schema.CollectionSchema schema = 3;
schema.CollectionSchema schema = 3 [deprecated = true]; // schema is managed by streaming node itself now, so it should not be passed by rpc.
repeated SegmentNotCreatedByStreaming segments_not_created_by_streaming = 4; // Should be flushed by streaming service when upgrading.
}

File diff suppressed because it is too large Load Diff

View File

@ -218,7 +218,6 @@ message SchemaChangeMessageHeader{
// SchemaChangeMessageBody is the body of CollectionSchema update message.
message SchemaChangeMessageBody{
schema.CollectionSchema schema = 1;
uint64 modify_ts = 2;
}
///

View File

@ -1668,8 +1668,7 @@ type SchemaChangeMessageBody struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Schema *schemapb.CollectionSchema `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"`
ModifyTs uint64 `protobuf:"varint,2,opt,name=modify_ts,json=modifyTs,proto3" json:"modify_ts,omitempty"`
Schema *schemapb.CollectionSchema `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"`
}
func (x *SchemaChangeMessageBody) Reset() {
@ -1711,13 +1710,6 @@ func (x *SchemaChangeMessageBody) GetSchema() *schemapb.CollectionSchema {
return nil
}
func (x *SchemaChangeMessageBody) GetModifyTs() uint64 {
if x != nil {
return x.ModifyTs
}
return 0
}
// ManualFlushExtraResponse is the extra response of manual flush message.
type ManualFlushExtraResponse struct {
state protoimpl.MessageState
@ -2229,97 +2221,95 @@ var file_messages_proto_rawDesc = []byte{
0x64, 0x12, 0x2e, 0x0a, 0x13, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x67,
0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x11,
0x66, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64,
0x73, 0x22, 0x75, 0x0a, 0x17, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x43, 0x68, 0x61, 0x6e, 0x67,
0x73, 0x22, 0x58, 0x0a, 0x17, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x43, 0x68, 0x61, 0x6e, 0x67,
0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x3d, 0x0a, 0x06,
0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d,
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65,
0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68,
0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1b, 0x0a, 0x09, 0x6d,
0x6f, 0x64, 0x69, 0x66, 0x79, 0x5f, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08,
0x6d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x54, 0x73, 0x22, 0x3b, 0x0a, 0x18, 0x4d, 0x61, 0x6e, 0x75,
0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f,
0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65,
0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, 0x5a, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6e, 0x74,
0x65, 0x78, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x74, 0x78, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
0x01, 0x28, 0x03, 0x52, 0x05, 0x74, 0x78, 0x6e, 0x49, 0x64, 0x12, 0x35, 0x0a, 0x16, 0x6b, 0x65,
0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63,
0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x6b, 0x65, 0x65, 0x70,
0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64,
0x73, 0x22, 0xc4, 0x01, 0x0a, 0x10, 0x52, 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61,
0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64,
0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x02,
0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x4d, 0x51,
0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x2e, 0x50, 0x72,
0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x70,
0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x72, 0x6f,
0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03,
0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14,
0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76,
0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9b, 0x01, 0x0a, 0x0f, 0x42, 0x72, 0x6f,
0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x21, 0x0a, 0x0c,
0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,
0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x49, 0x64, 0x12,
0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03,
0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x47, 0x0a,
0x0d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x03,
0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72,
0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, 0x5e, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72,
0x63, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x3d, 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18,
0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65,
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x52, 0x06, 0x64, 0x6f,
0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28,
0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22, 0x88, 0x01, 0x0a, 0x0c, 0x43, 0x69, 0x70, 0x68, 0x65,
0x72, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x13, 0x0a, 0x05, 0x65, 0x7a, 0x5f, 0x69, 0x64,
0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x65, 0x7a, 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d,
0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20,
0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49,
0x64, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x61, 0x66, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20,
0x01, 0x28, 0x0c, 0x52, 0x07, 0x73, 0x61, 0x66, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x23, 0x0a, 0x0d,
0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x04, 0x20,
0x01, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x42, 0x79, 0x74, 0x65,
0x73, 0x2a, 0x9a, 0x02, 0x0a, 0x0b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70,
0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c,
0x0a, 0x08, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06,
0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x65, 0x6c, 0x65,
0x74, 0x65, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x04, 0x12,
0x14, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74,
0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x12, 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c,
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f, 0x43, 0x72, 0x65,
0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, 0x11,
0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10,
0x08, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68,
0x10, 0x09, 0x12, 0x11, 0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d,
0x65, 0x6e, 0x74, 0x10, 0x0a, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x10,
0x0b, 0x12, 0x10, 0x0a, 0x0c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x43, 0x68, 0x61, 0x6e, 0x67,
0x65, 0x10, 0x0c, 0x12, 0x0d, 0x0a, 0x08, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x10,
0x84, 0x07, 0x12, 0x0e, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x10,
0x85, 0x07, 0x12, 0x10, 0x0a, 0x0b, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78,
0x6e, 0x10, 0x86, 0x07, 0x12, 0x08, 0x0a, 0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7, 0x07, 0x2a, 0x74,
0x0a, 0x08, 0x54, 0x78, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x78,
0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78,
0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x54,
0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x10, 0x02, 0x12, 0x10, 0x0a, 0x0c,
0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x03, 0x12, 0x11,
0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x10,
0x04, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b,
0x65, 0x64, 0x10, 0x05, 0x2a, 0x6c, 0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x19, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72,
0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10,
0x00, 0x12, 0x1d, 0x0a, 0x19, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d,
0x61, 0x69, 0x6e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x49, 0x44, 0x10, 0x01,
0x12, 0x20, 0x0a, 0x1c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61,
0x69, 0x6e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65,
0x10, 0x02, 0x42, 0x35, 0x5a, 0x33, 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, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6d,
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x33,
0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0x3b, 0x0a, 0x18, 0x4d,
0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x45, 0x78, 0x74, 0x72, 0x61, 0x52,
0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65,
0x6e, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65,
0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, 0x5a, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x43,
0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x74, 0x78, 0x6e, 0x5f, 0x69, 0x64,
0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x74, 0x78, 0x6e, 0x49, 0x64, 0x12, 0x35, 0x0a,
0x16, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69,
0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x6b,
0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63,
0x6f, 0x6e, 0x64, 0x73, 0x22, 0xc4, 0x01, 0x0a, 0x10, 0x52, 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73,
0x61, 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79,
0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c,
0x6f, 0x61, 0x64, 0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65,
0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e,
0x52, 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74,
0x2e, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79,
0x52, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f,
0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12,
0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65,
0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09,
0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9b, 0x01, 0x0a, 0x0f,
0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12,
0x21, 0x0a, 0x0c, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18,
0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74,
0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18,
0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73,
0x12, 0x47, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79,
0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e,
0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0c, 0x52, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, 0x5e, 0x0a, 0x0b, 0x52, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x3d, 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61,
0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73,
0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x52,
0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02,
0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22, 0x88, 0x01, 0x0a, 0x0c, 0x43, 0x69,
0x70, 0x68, 0x65, 0x72, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x13, 0x0a, 0x05, 0x65, 0x7a,
0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x65, 0x7a, 0x49, 0x64, 0x12,
0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64,
0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69,
0x6f, 0x6e, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x61, 0x66, 0x65, 0x5f, 0x6b, 0x65, 0x79,
0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x73, 0x61, 0x66, 0x65, 0x4b, 0x65, 0x79, 0x12,
0x23, 0x0a, 0x0d, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73,
0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x42,
0x79, 0x74, 0x65, 0x73, 0x2a, 0x9a, 0x02, 0x0a, 0x0b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10,
0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x10, 0x01, 0x12,
0x0a, 0x0a, 0x06, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44,
0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68,
0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c,
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x12, 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70,
0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f,
0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10,
0x07, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69,
0x6f, 0x6e, 0x10, 0x08, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c,
0x75, 0x73, 0x68, 0x10, 0x09, 0x12, 0x11, 0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53,
0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x10, 0x0a, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6d, 0x70, 0x6f,
0x72, 0x74, 0x10, 0x0b, 0x12, 0x10, 0x0a, 0x0c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x43, 0x68,
0x61, 0x6e, 0x67, 0x65, 0x10, 0x0c, 0x12, 0x0d, 0x0a, 0x08, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54,
0x78, 0x6e, 0x10, 0x84, 0x07, 0x12, 0x0e, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54,
0x78, 0x6e, 0x10, 0x85, 0x07, 0x12, 0x10, 0x0a, 0x0b, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63,
0x6b, 0x54, 0x78, 0x6e, 0x10, 0x86, 0x07, 0x12, 0x08, 0x0a, 0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7,
0x07, 0x2a, 0x74, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a,
0x0a, 0x54, 0x78, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0f, 0x0a,
0x0b, 0x54, 0x78, 0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x10, 0x01, 0x12, 0x0f,
0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x10, 0x02, 0x12,
0x10, 0x0a, 0x0c, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x10,
0x03, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61,
0x63, 0x6b, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62,
0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0x05, 0x2a, 0x6c, 0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75,
0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x19, 0x0a, 0x15, 0x52, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f,
0x77, 0x6e, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x49,
0x44, 0x10, 0x01, 0x12, 0x20, 0x0a, 0x1c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44,
0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e,
0x61, 0x6d, 0x65, 0x10, 0x02, 0x42, 0x35, 0x5a, 0x33, 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, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x33,
}
var (

View File

@ -6,6 +6,7 @@ option go_package = "github.com/milvus-io/milvus/pkg/v2/proto/streamingpb";
import "messages.proto";
import "milvus.proto";
import "schema.proto";
import "google/protobuf/empty.proto";
import "google/protobuf/any.proto";
@ -528,6 +529,14 @@ message VChannelMeta {
message CollectionInfoOfVChannel {
int64 collection_id = 1; // collection id.
repeated PartitionInfoOfVChannel partitions = 2; // partitions.
repeated CollectionSchemaOfVChannel schemas = 3; // The schemas of the vchannel.
}
// CollectionSchemaOfVChannel is the collection schema in vchannel.
message CollectionSchemaOfVChannel {
schema.CollectionSchema schema = 1; // The schemas of the vchannel.
uint64 checkpoint_time_tick = 2; // The timetick of the schema changed, also the version of the schema.
VChannelSchemaState state = 3; // The state of the schema.
}
// PartitionInfoOfVChannel is the partition info in vchannel.
@ -543,6 +552,13 @@ enum VChannelState {
// VCHANNEL_STATE_SPLITTED = 3; // TODO: vchannel is splitted to other vchannels, used to support shard-splitting.
}
// VChannelSchemaState is the state of vchannel schema.
enum VChannelSchemaState {
VCHANNEL_SCHEMA_STATE_UNKNOWN = 0; // should never used.
VCHANNEL_SCHEMA_STATE_NORMAL = 1; // vchannel schema is normal.
VCHANNEL_SCHEMA_STATE_DROPPED = 2; // vchannel schema is dropped.
}
///
/// SegmentAssignment
///

File diff suppressed because it is too large Load Diff

View File

@ -222,7 +222,7 @@ func mustAsSpecializedMutableMessage[H proto.Message, B proto.Message](msg Basic
smsg, err := asSpecializedMutableMessage[H, B](msg)
if err != nil {
panic(
fmt.Sprintf("failed to parse immutable message: %s @ %s, %d, %d",
fmt.Sprintf("failed to parse mutable message: %s @ %s, %d, %d",
err.Error(),
msg.MessageType(),
msg.TimeTick(),
@ -236,6 +236,9 @@ func mustAsSpecializedMutableMessage[H proto.Message, B proto.Message](msg Basic
// Return nil, error if the message is the target specialized message but failed to decode the specialized header.
// Return specializedMutableMessage, nil if the message is the target specialized message and successfully decoded the specialized header.
func asSpecializedMutableMessage[H proto.Message, B proto.Message](msg BasicMessage) (specializedMutableMessage[H, B], error) {
if already, ok := msg.(specializedMutableMessage[H, B]); ok {
return already, nil
}
underlying := msg.(*messageImpl)
var header H
@ -289,6 +292,9 @@ func mustAsSpecializedImmutableMessage[H proto.Message, B proto.Message](msg Imm
// Return nil, error if the message is the target specialized message but failed to decode the specialized header.
// Return asSpecializedImmutableMessage, nil if the message is the target specialized message and successfully decoded the specialized header.
func asSpecializedImmutableMessage[H proto.Message, B proto.Message](msg ImmutableMessage) (specializedImmutableMessage[H, B], error) {
if already, ok := msg.(specializedImmutableMessage[H, B]); ok {
return already, nil
}
underlying, ok := msg.(*immutableMessageImpl)
if !ok {
// maybe a txn message.

View File

@ -36,6 +36,9 @@ func TestAsSpecializedMessage(t *testing.T) {
assert.NoError(t, err)
assert.Equal(t, int64(1), body.CollectionID)
_, err = message.AsMutableInsertMessageV1(insertMsg)
assert.NoError(t, err)
h := insertMsg.Header()
h.Partitions[0].SegmentAssignment = &message.SegmentAssignment{
SegmentId: 1,
@ -64,6 +67,10 @@ func TestAsSpecializedMessage(t *testing.T) {
assert.NoError(t, err)
assert.Equal(t, int64(1), body.CollectionID)
// double as is ok.
_, err = message.AsImmutableInsertMessageV1(insertMsg2)
assert.NoError(t, err)
insertMsg2 = message.MustAsImmutableInsertMessageV1(m2)
assert.NotNil(t, insertMsg2)