mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-06 17:18:35 +08:00
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:
parent
9fbd41a97d
commit
e9ab73e93d
@ -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)))
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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())
|
||||
}
|
||||
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
}
|
||||
@ -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"},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
@ -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) {
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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() {
|
||||
|
||||
@ -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)
|
||||
|
||||
22
internal/flushcommon/metacache/schema.go
Normal file
22
internal/flushcommon/metacache/schema.go
Normal 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
|
||||
}
|
||||
18
internal/flushcommon/metacache/schema_test.go
Normal file
18
internal/flushcommon/metacache/schema_test.go
Normal 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))
|
||||
}
|
||||
@ -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 {
|
||||
|
||||
@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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())
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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(
|
||||
|
||||
@ -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 }
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -6,6 +6,7 @@ const (
|
||||
DirectoryWAL = "wal"
|
||||
DirectorySegmentAssign = "segment-assign"
|
||||
DirectoryVChannel = "vchannel"
|
||||
DirectorySchema = "schema"
|
||||
|
||||
KeyConsumeCheckpoint = "consume-checkpoint"
|
||||
)
|
||||
|
||||
@ -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) + "/"
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
@ -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 {
|
||||
|
||||
@ -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(
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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{
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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{
|
||||
|
||||
@ -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
@ -218,7 +218,6 @@ message SchemaChangeMessageHeader{
|
||||
// SchemaChangeMessageBody is the body of CollectionSchema update message.
|
||||
message SchemaChangeMessageBody{
|
||||
schema.CollectionSchema schema = 1;
|
||||
uint64 modify_ts = 2;
|
||||
}
|
||||
|
||||
///
|
||||
|
||||
@ -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 (
|
||||
|
||||
@ -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
@ -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.
|
||||
|
||||
@ -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)
|
||||
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user