feat: support to replicate import msg (#39171)

- issue: #39849

---------

Signed-off-by: SimFG <bang.fu@zilliz.com>
Signed-off-by: chyezh <chyezh@outlook.com>
Co-authored-by: chyezh <chyezh@outlook.com>
This commit is contained in:
SimFG 2025-02-16 00:08:13 +08:00 committed by GitHub
parent b59555057d
commit 047254665d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
69 changed files with 4260 additions and 1899 deletions

View File

@ -221,7 +221,7 @@ jobs:
name: Integration Test
needs: Build
runs-on: ubuntu-latest
timeout-minutes: 120
timeout-minutes: 150
steps:
- name: Maximize build space
uses: easimon/maximize-build-space@master

6
go.mod
View File

@ -18,12 +18,12 @@ require (
github.com/gin-gonic/gin v1.9.1
github.com/go-playground/validator/v10 v10.14.0
github.com/gofrs/flock v0.8.1
github.com/golang/protobuf v1.5.4
github.com/golang/protobuf v1.5.4 // indirect
github.com/google/btree v1.1.2
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0
github.com/klauspost/compress v1.17.9
github.com/mgutz/ansi v0.0.0-20200706080929-d51e80ef957d
github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250208062437-5af22aa4b559
github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250214033407-ad08272e542b
github.com/minio/minio-go/v7 v7.0.73
github.com/pingcap/log v1.1.1-0.20221015072633-39906604fb81
github.com/prometheus/client_golang v1.14.0
@ -72,6 +72,7 @@ require (
github.com/greatroar/blobloom v0.0.0-00010101000000-000000000000
github.com/hashicorp/golang-lru/v2 v2.0.7
github.com/jolestar/go-commons-pool/v2 v2.1.2
github.com/magiconair/properties v1.8.5
github.com/milvus-io/milvus/pkg v0.0.2-0.20241126032235-cb6542339e84
github.com/pkg/errors v0.9.1
github.com/remeh/sizedwaitgroup v1.0.0
@ -181,7 +182,6 @@ require (
github.com/leodido/go-urn v1.2.4 // indirect
github.com/linkedin/goavro/v2 v2.11.1 // indirect
github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect
github.com/magiconair/properties v1.8.5 // indirect
github.com/mattn/go-colorable v0.1.11 // indirect
github.com/mattn/go-isatty v0.0.19 // indirect
github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect

4
go.sum
View File

@ -660,8 +660,8 @@ github.com/milvus-io/cgosymbolizer v0.0.0-20240722103217-b7dee0e50119 h1:9VXijWu
github.com/milvus-io/cgosymbolizer v0.0.0-20240722103217-b7dee0e50119/go.mod h1:DvXTE/K/RtHehxU8/GtDs4vFtfw64jJ3PaCnFri8CRg=
github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b h1:TfeY0NxYxZzUfIfYe5qYDBzt4ZYRqzUjTR6CvUzjat8=
github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b/go.mod h1:iwW+9cWfIzzDseEBCCeDSN5SD16Tidvy8cwQ7ZY8Qj4=
github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250208062437-5af22aa4b559 h1:c8n10eBkYU/HYaDUNAaKog4aIA3ZHO+GL7bHN2Ug/MA=
github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250208062437-5af22aa4b559/go.mod h1:/6UT4zZl6awVeXLeE7UGDWZvXj3IWkRsh3mqsn0DiAs=
github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250214033407-ad08272e542b h1:s3gdV+iYJMvo9mydBJVAEA2Uaz29eIuUnQK867U3G8I=
github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250214033407-ad08272e542b/go.mod h1:/6UT4zZl6awVeXLeE7UGDWZvXj3IWkRsh3mqsn0DiAs=
github.com/milvus-io/pulsar-client-go v0.12.1 h1:O2JZp1tsYiO7C0MQ4hrUY/aJXnn2Gry6hpm7UodghmE=
github.com/milvus-io/pulsar-client-go v0.12.1/go.mod h1:dkutuH4oS2pXiGm+Ti7fQZ4MRjrMPZ8IJeEGAWMeckk=
github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 h1:AMFGa4R4MiIpspGNG7Z948v4n35fFGB3RR3G/ry4FWs=

View File

@ -9,6 +9,7 @@ packages:
interfaces:
WALAccesser:
Utility:
Broadcast:
github.com/milvus-io/milvus/internal/streamingcoord/server/balancer:
interfaces:
Balancer:
@ -38,9 +39,6 @@ packages:
github.com/milvus-io/milvus/internal/streamingnode/client/handler/consumer:
interfaces:
Consumer:
github.com/milvus-io/milvus/internal/streamingnode/server/flusher:
interfaces:
FlushMsgHandler:
github.com/milvus-io/milvus/internal/streamingnode/server/wal:
interfaces:
OpenerBuilder:
@ -91,6 +89,9 @@ packages:
github.com/milvus-io/milvus/internal/util/searchutil/optimizers:
interfaces:
QueryHook:
# github.com/milvus-io/milvus/internal/flushcommon/util:
# interfaces:
# MsgHandler:
google.golang.org/grpc/resolver:
interfaces:
ClientConn:

View File

@ -61,6 +61,7 @@ type compactionPlanContext interface {
removeTasksByChannel(channel string)
setTaskScheduler(scheduler *taskScheduler)
checkAndSetSegmentStating(channel string, segmentID int64) bool
getCompactionTasksNum(filters ...compactionTaskFilter) int
}
var (
@ -146,7 +147,7 @@ func (sna *SlotBasedNodeAssigner) pickAnyNode(task CompactionTask) (nodeID int64
}
type compactionPlanHandler struct {
queueTasks CompactionQueue
queueTasks *CompactionQueue
executingGuard lock.RWMutex
executingTasks map[int64]CompactionTask // planID -> task
@ -271,7 +272,7 @@ func newCompactionPlanHandler(cluster Cluster, sessions session.DataNodeManager,
// TODO[GOOSE]: Higher capacity makes tasks waiting longer, which need to be get rid of.
capacity := paramtable.Get().DataCoordCfg.CompactionTaskQueueCapacity.GetAsInt()
return &compactionPlanHandler{
queueTasks: *NewCompactionQueue(capacity, getPrioritizer()),
queueTasks: NewCompactionQueue(capacity, getPrioritizer()),
meta: meta,
sessions: sessions,
allocator: allocator,
@ -806,6 +807,45 @@ func (c *compactionPlanHandler) checkDelay(t CompactionTask) {
}
}
func (c *compactionPlanHandler) getCompactionTasksNum(filters ...compactionTaskFilter) int {
cnt := 0
isMatch := func(task CompactionTask) bool {
for _, f := range filters {
if !f(task) {
return false
}
}
return true
}
c.queueTasks.ForEach(func(task CompactionTask) {
if isMatch(task) {
cnt += 1
}
})
c.executingGuard.RLock()
for _, t := range c.executingTasks {
if isMatch(t) {
cnt += 1
}
}
c.executingGuard.RUnlock()
return cnt
}
type compactionTaskFilter func(task CompactionTask) bool
func CollectionIDCompactionTaskFilter(collectionID int64) compactionTaskFilter {
return func(task CompactionTask) bool {
return task.GetTaskProto().GetCollectionID() == collectionID
}
}
func L0CompactionCompactionTaskFilter() compactionTaskFilter {
return func(task CompactionTask) bool {
return task.GetTaskProto().GetType() == datapb.CompactionType_Level0DeleteCompaction
}
}
var (
ioPool *conc.Pool[any]
ioPoolInitOnce sync.Once

View File

@ -18,12 +18,17 @@ type l0CompactionPolicy struct {
meta *meta
activeCollections *activeCollections
// key: collectionID, value: reference count
skipCompactionCollections map[int64]int
skipLocker sync.RWMutex
}
func newL0CompactionPolicy(meta *meta) *l0CompactionPolicy {
return &l0CompactionPolicy{
meta: meta,
activeCollections: newActiveCollections(),
meta: meta,
activeCollections: newActiveCollections(),
skipCompactionCollections: make(map[int64]int),
}
}
@ -31,6 +36,34 @@ func (policy *l0CompactionPolicy) Enable() bool {
return Params.DataCoordCfg.EnableAutoCompaction.GetAsBool()
}
func (policy *l0CompactionPolicy) AddSkipCollection(collectionID UniqueID) {
policy.skipLocker.Lock()
defer policy.skipLocker.Unlock()
if _, ok := policy.skipCompactionCollections[collectionID]; !ok {
policy.skipCompactionCollections[collectionID] = 1
} else {
policy.skipCompactionCollections[collectionID]++
}
}
func (policy *l0CompactionPolicy) RemoveSkipCollection(collectionID UniqueID) {
policy.skipLocker.Lock()
defer policy.skipLocker.Unlock()
refCount := policy.skipCompactionCollections[collectionID]
if refCount > 1 {
policy.skipCompactionCollections[collectionID]--
} else {
delete(policy.skipCompactionCollections, collectionID)
}
}
func (policy *l0CompactionPolicy) isSkipCollection(collectionID UniqueID) bool {
policy.skipLocker.RLock()
defer policy.skipLocker.RUnlock()
return policy.skipCompactionCollections[collectionID] > 0
}
// Notify policy to record the active updated(when adding a new L0 segment) collections.
func (policy *l0CompactionPolicy) OnCollectionUpdate(collectionID int64) {
policy.activeCollections.Record(collectionID)
@ -50,8 +83,11 @@ func (policy *l0CompactionPolicy) Trigger() (events map[CompactionTriggerType][]
idleCollsSet := typeutil.NewUniqueSet(idleColls...)
activeL0Views, idleL0Views := []CompactionView{}, []CompactionView{}
for collID, segments := range latestCollSegs {
policy.activeCollections.Read(collID)
if policy.isSkipCollection(collID) {
continue
}
policy.activeCollections.Read(collID)
levelZeroSegments := lo.Filter(segments, func(info *SegmentInfo, _ int) bool {
return info.GetLevel() == datapb.SegmentLevel_L0
})

View File

@ -113,6 +113,30 @@ func (s *L0CompactionPolicySuite) TestTriggerIdle() {
for _, view := range cView.GetSegmentsView() {
s.Equal(datapb.SegmentLevel_L0, view.Level)
}
// test for skip collection
s.l0_policy.AddSkipCollection(1)
s.l0_policy.AddSkipCollection(1)
// Test for skip collection
events, err = s.l0_policy.Trigger()
s.NoError(err)
s.Empty(events)
// Test for skip collection with ref count
s.l0_policy.RemoveSkipCollection(1)
events, err = s.l0_policy.Trigger()
s.NoError(err)
s.Empty(events)
s.l0_policy.RemoveSkipCollection(1)
events, err = s.l0_policy.Trigger()
s.NoError(err)
s.Equal(1, len(events))
gotViews, ok = events[TriggerTypeLevelZeroViewIDLE]
s.True(ok)
s.NotNil(gotViews)
s.Equal(1, len(gotViews))
log.Info("cView", zap.String("string", cView.String()))
}

View File

@ -22,6 +22,7 @@ import (
"time"
"github.com/cockroachdb/errors"
"github.com/magiconair/properties/assert"
"github.com/samber/lo"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
@ -1187,3 +1188,60 @@ func TestCheckDelay(t *testing.T) {
}, nil, nil, nil, nil, nil)
handler.checkDelay(t3)
}
func TestGetCompactionTasksNum(t *testing.T) {
queueTasks := NewCompactionQueue(10, DefaultPrioritizer)
queueTasks.Enqueue(
newMixCompactionTask(&datapb.CompactionTask{
StartTime: time.Now().Add(-100 * time.Minute).Unix(),
CollectionID: 1,
Type: datapb.CompactionType_MixCompaction,
}, nil, nil, nil),
)
queueTasks.Enqueue(
newL0CompactionTask(&datapb.CompactionTask{
StartTime: time.Now().Add(-100 * time.Minute).Unix(),
CollectionID: 1,
Type: datapb.CompactionType_Level0DeleteCompaction,
}, nil, nil, nil),
)
queueTasks.Enqueue(
newClusteringCompactionTask(&datapb.CompactionTask{
StartTime: time.Now().Add(-100 * time.Minute).Unix(),
CollectionID: 10,
Type: datapb.CompactionType_ClusteringCompaction,
}, nil, nil, nil, nil, nil),
)
executingTasks := make(map[int64]CompactionTask, 0)
executingTasks[1] = newMixCompactionTask(&datapb.CompactionTask{
StartTime: time.Now().Add(-100 * time.Minute).Unix(),
CollectionID: 1,
Type: datapb.CompactionType_MixCompaction,
}, nil, nil, nil)
executingTasks[2] = newL0CompactionTask(&datapb.CompactionTask{
StartTime: time.Now().Add(-100 * time.Minute).Unix(),
CollectionID: 10,
Type: datapb.CompactionType_Level0DeleteCompaction,
}, nil, nil, nil)
handler := &compactionPlanHandler{
queueTasks: queueTasks,
executingTasks: executingTasks,
}
t.Run("no filter", func(t *testing.T) {
i := handler.getCompactionTasksNum()
assert.Equal(t, 5, i)
})
t.Run("collection id filter", func(t *testing.T) {
i := handler.getCompactionTasksNum(CollectionIDCompactionTaskFilter(1))
assert.Equal(t, 3, i)
})
t.Run("l0 compaction filter", func(t *testing.T) {
i := handler.getCompactionTasksNum(L0CompactionCompactionTaskFilter())
assert.Equal(t, 2, i)
})
t.Run("collection id and l0 compaction filter", func(t *testing.T) {
i := handler.getCompactionTasksNum(CollectionIDCompactionTaskFilter(1), L0CompactionCompactionTaskFilter())
assert.Equal(t, 1, i)
})
}

View File

@ -50,6 +50,11 @@ type spyCompactionHandler struct {
meta *meta
}
// getCompactionTasksNum implements compactionPlanContext.
func (h *spyCompactionHandler) getCompactionTasksNum(filters ...compactionTaskFilter) int {
return 0
}
func (h *spyCompactionHandler) getCompactionTasksNumBySignalID(signalID int64) int {
return 0
}

View File

@ -27,6 +27,7 @@ import (
"github.com/milvus-io/milvus/internal/datacoord/allocator"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/proto/datapb"
"github.com/milvus-io/milvus/pkg/proto/internalpb"
"github.com/milvus-io/milvus/pkg/util/logutil"
)
@ -62,6 +63,8 @@ type TriggerManager interface {
Stop()
OnCollectionUpdate(collectionID int64)
ManualTrigger(ctx context.Context, collectionID int64, clusteringCompaction bool) (UniqueID, error)
GetPauseCompactionChan(jobID, collectionID int64) <-chan struct{}
GetResumeCompactionChan(jobID, collectionID int64) <-chan struct{}
}
var _ TriggerManager = (*CompactionTriggerManager)(nil)
@ -82,21 +85,35 @@ type CompactionTriggerManager struct {
allocator allocator.Allocator
meta *meta
imeta ImportMeta
l0Policy *l0CompactionPolicy
clusteringPolicy *clusteringCompactionPolicy
singlePolicy *singleCompactionPolicy
cancel context.CancelFunc
closeWg sync.WaitGroup
l0Triggering bool
l0SigLock *sync.Mutex
l0TickSig *sync.Cond
pauseCompactionChanMap map[int64]chan struct{}
resumeCompactionChanMap map[int64]chan struct{}
compactionChanLock sync.Mutex
}
func NewCompactionTriggerManager(alloc allocator.Allocator, handler Handler, compactionHandler compactionPlanContext, meta *meta) *CompactionTriggerManager {
func NewCompactionTriggerManager(alloc allocator.Allocator, handler Handler, compactionHandler compactionPlanContext, meta *meta, imeta ImportMeta) *CompactionTriggerManager {
m := &CompactionTriggerManager{
allocator: alloc,
handler: handler,
compactionHandler: compactionHandler,
meta: meta,
allocator: alloc,
handler: handler,
compactionHandler: compactionHandler,
meta: meta,
imeta: imeta,
pauseCompactionChanMap: make(map[int64]chan struct{}),
resumeCompactionChanMap: make(map[int64]chan struct{}),
}
m.l0SigLock = &sync.Mutex{}
m.l0TickSig = sync.NewCond(m.l0SigLock)
m.l0Policy = newL0CompactionPolicy(meta)
m.clusteringPolicy = newClusteringCompactionPolicy(meta, m.allocator, m.handler)
m.singlePolicy = newSingleCompactionPolicy(meta, m.allocator, m.handler)
@ -126,6 +143,64 @@ func (m *CompactionTriggerManager) Stop() {
m.closeWg.Wait()
}
func (m *CompactionTriggerManager) pauseL0SegmentCompacting(jobID, collectionID int64) {
m.l0Policy.AddSkipCollection(collectionID)
m.l0SigLock.Lock()
for m.l0Triggering {
m.l0TickSig.Wait()
}
m.l0SigLock.Unlock()
m.compactionChanLock.Lock()
if ch, ok := m.pauseCompactionChanMap[jobID]; ok {
close(ch)
}
m.compactionChanLock.Unlock()
}
func (m *CompactionTriggerManager) resumeL0SegmentCompacting(jobID, collectionID int64) {
m.compactionChanLock.Lock()
m.l0Policy.RemoveSkipCollection(collectionID)
if ch, ok := m.resumeCompactionChanMap[jobID]; ok {
close(ch)
delete(m.pauseCompactionChanMap, jobID)
delete(m.resumeCompactionChanMap, jobID)
}
m.compactionChanLock.Unlock()
}
func (m *CompactionTriggerManager) GetPauseCompactionChan(jobID, collectionID int64) <-chan struct{} {
m.compactionChanLock.Lock()
defer m.compactionChanLock.Unlock()
if ch, ok := m.pauseCompactionChanMap[jobID]; ok {
return ch
}
ch := make(chan struct{})
m.pauseCompactionChanMap[jobID] = ch
go m.pauseL0SegmentCompacting(jobID, collectionID)
return ch
}
func (m *CompactionTriggerManager) GetResumeCompactionChan(jobID, collectionID int64) <-chan struct{} {
m.compactionChanLock.Lock()
defer m.compactionChanLock.Unlock()
if ch, ok := m.resumeCompactionChanMap[jobID]; ok {
return ch
}
ch := make(chan struct{})
m.resumeCompactionChanMap[jobID] = ch
go m.resumeL0SegmentCompacting(jobID, collectionID)
return ch
}
func (m *CompactionTriggerManager) setL0Triggering(b bool) {
m.l0SigLock.Lock()
defer m.l0SigLock.Unlock()
m.l0Triggering = b
if !b {
m.l0TickSig.Broadcast()
}
}
func (m *CompactionTriggerManager) loop(ctx context.Context) {
defer logutil.LogPanic()
@ -150,9 +225,11 @@ func (m *CompactionTriggerManager) loop(ctx context.Context) {
log.RatedInfo(10, "Skip trigger l0 compaction since compactionHandler is full")
continue
}
m.setL0Triggering(true)
events, err := m.l0Policy.Trigger()
if err != nil {
log.Warn("Fail to trigger L0 policy", zap.Error(err))
m.setL0Triggering(false)
continue
}
if len(events) > 0 {
@ -160,6 +237,7 @@ func (m *CompactionTriggerManager) loop(ctx context.Context) {
m.notify(ctx, triggerType, views)
}
}
m.setL0Triggering(false)
case <-clusteringTicker.C:
if !m.clusteringPolicy.Enable() {
continue
@ -262,6 +340,12 @@ func (m *CompactionTriggerManager) SubmitL0ViewToScheduler(ctx context.Context,
return
}
err = m.addL0ImportTaskForImport(ctx, collection, view)
if err != nil {
log.Warn("Failed to submit compaction view to scheduler because add l0 import task fail", zap.Error(err))
return
}
task := &datapb.CompactionTask{
TriggerID: taskID, // inner trigger, use task id as trigger id
PlanID: taskID,
@ -294,6 +378,75 @@ func (m *CompactionTriggerManager) SubmitL0ViewToScheduler(ctx context.Context,
)
}
func (m *CompactionTriggerManager) addL0ImportTaskForImport(ctx context.Context, collection *collectionInfo, view CompactionView) error {
// add l0 import task for the collection if the collection is importing
importJobs := m.imeta.GetJobBy(ctx, WithCollectionID(collection.ID), WithoutJobStates(internalpb.ImportJobState_Completed, internalpb.ImportJobState_Failed))
if len(importJobs) > 0 {
partitionID := view.GetGroupLabel().PartitionID
var (
fileSize int64 = 0
totalRows int64 = 0
totalMemorySize int64 = 0
importPaths []string
)
idStart := time.Now().UnixMilli()
for _, segmentView := range view.GetSegmentsView() {
segInfo := m.meta.GetSegment(ctx, segmentView.ID)
if segInfo == nil {
continue
}
totalRows += int64(segmentView.DeltaRowCount)
totalMemorySize += int64(segmentView.DeltaSize)
for _, deltaLogs := range segInfo.GetDeltalogs() {
for _, binlog := range deltaLogs.GetBinlogs() {
fileSize += binlog.GetLogSize()
importPaths = append(importPaths, binlog.GetLogPath())
}
}
}
for i, job := range importJobs {
newTasks, err := NewImportTasks([][]*datapb.ImportFileStats{
{
{
ImportFile: &internalpb.ImportFile{
Id: idStart + int64(i),
Paths: importPaths,
},
FileSize: fileSize,
TotalRows: totalRows,
TotalMemorySize: totalMemorySize,
HashedStats: map[string]*datapb.PartitionImportStats{
// which is vchannel
view.GetGroupLabel().Channel: {
PartitionRows: map[int64]int64{
partitionID: totalRows,
},
PartitionDataSize: map[int64]int64{
partitionID: totalMemorySize,
},
},
},
},
},
}, job, m.allocator, m.meta)
if err != nil {
log.Warn("new import tasks failed", zap.Error(err))
return err
}
for _, t := range newTasks {
err = m.imeta.AddTask(ctx, t)
if err != nil {
log.Warn("add new l0 import task from l0 compaction failed", WrapTaskLog(t, zap.Error(err))...)
return err
}
log.Info("add new l0 import task from l0 compaction", WrapTaskLog(t)...)
}
}
}
return nil
}
func (m *CompactionTriggerManager) SubmitClusteringViewToScheduler(ctx context.Context, view CompactionView) {
log := log.Ctx(ctx).With(zap.String("view", view.String()))
taskID, _, err := m.allocator.AllocN(2)

View File

@ -4,8 +4,10 @@ import (
"context"
"strconv"
"testing"
"time"
"github.com/samber/lo"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
"go.uber.org/zap"
@ -13,10 +15,12 @@ import (
"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/datacoord/allocator"
"github.com/milvus-io/milvus/internal/metastore/mocks"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/proto/datapb"
"github.com/milvus-io/milvus/pkg/proto/internalpb"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)
@ -32,6 +36,7 @@ type CompactionTriggerManagerSuite struct {
mockPlanContext *MockCompactionPlanContext
testLabel *CompactionGroupLabel
meta *meta
imeta ImportMeta
triggerManager *CompactionTriggerManager
}
@ -51,8 +56,14 @@ func (s *CompactionTriggerManagerSuite) SetupTest() {
for id, segment := range segments {
s.meta.segments.SetSegment(id, segment)
}
s.triggerManager = NewCompactionTriggerManager(s.mockAlloc, s.handler, s.mockPlanContext, s.meta)
catalog := mocks.NewDataCoordCatalog(s.T())
catalog.EXPECT().ListPreImportTasks(mock.Anything).Return([]*datapb.PreImportTask{}, nil)
catalog.EXPECT().ListImportTasks(mock.Anything).Return([]*datapb.ImportTaskV2{}, nil)
catalog.EXPECT().ListImportJobs(mock.Anything).Return([]*datapb.ImportJob{}, nil)
importMeta, err := NewImportMeta(context.TODO(), catalog)
s.Require().NoError(err)
s.imeta = importMeta
s.triggerManager = NewCompactionTriggerManager(s.mockAlloc, s.handler, s.mockPlanContext, s.meta, s.imeta)
}
func (s *CompactionTriggerManagerSuite) TestNotifyByViewIDLE() {
@ -308,3 +319,73 @@ func (s *CompactionTriggerManagerSuite) TestGetExpectedSegmentSize() {
s.Equal(int64(100*1024*1024), getExpectedSegmentSize(s.triggerManager.meta, collection))
})
}
func TestCompactionAndImport(t *testing.T) {
paramtable.Init()
mockAlloc := allocator.NewMockAllocator(t)
handler := NewNMockHandler(t)
handler.EXPECT().GetCollection(mock.Anything, mock.Anything).Return(&collectionInfo{
ID: 1,
}, nil)
mockPlanContext := NewMockCompactionPlanContext(t)
mockPlanContext.EXPECT().isFull().Return(false)
testLabel := &CompactionGroupLabel{
CollectionID: 1,
PartitionID: 10,
Channel: "ch-1",
}
segments := genSegmentsForMeta(testLabel)
catelog := mocks.NewDataCoordCatalog(t)
catelog.EXPECT().AddSegment(mock.Anything, mock.Anything).Return(nil)
meta := &meta{
segments: NewSegmentsInfo(),
catalog: catelog,
}
for id, segment := range segments {
meta.segments.SetSegment(id, segment)
}
catalog := mocks.NewDataCoordCatalog(t)
catalog.EXPECT().ListPreImportTasks(mock.Anything).Return([]*datapb.PreImportTask{}, nil)
catalog.EXPECT().ListImportTasks(mock.Anything).Return([]*datapb.ImportTaskV2{}, nil)
catalog.EXPECT().ListImportJobs(mock.Anything).Return([]*datapb.ImportJob{
{
JobID: 100,
CollectionID: 1,
State: internalpb.ImportJobState_Importing,
},
}, nil).Once()
catalog.EXPECT().SaveImportTask(mock.Anything, mock.Anything).Return(nil)
importMeta, err := NewImportMeta(context.TODO(), catalog)
assert.NoError(t, err)
imeta := importMeta
triggerManager := NewCompactionTriggerManager(mockAlloc, handler, mockPlanContext, meta, imeta)
Params.Save(Params.DataCoordCfg.L0CompactionTriggerInterval.Key, "1")
defer Params.Reset(Params.DataCoordCfg.L0CompactionTriggerInterval.Key)
Params.Save(Params.DataCoordCfg.ClusteringCompactionTriggerInterval.Key, "6000000")
defer Params.Reset(Params.DataCoordCfg.ClusteringCompactionTriggerInterval.Key)
Params.Save(Params.DataCoordCfg.MixCompactionTriggerInterval.Key, "6000000")
defer Params.Reset(Params.DataCoordCfg.MixCompactionTriggerInterval.Key)
mockAlloc.EXPECT().AllocID(mock.Anything).Return(1, nil)
mockAlloc.EXPECT().AllocN(mock.Anything).Return(195300, 195300, nil)
mockAlloc.EXPECT().AllocTimestamp(mock.Anything).Return(30000, nil)
mockPlanContext.EXPECT().enqueueCompaction(mock.Anything).
RunAndReturn(func(task *datapb.CompactionTask) error {
assert.Equal(t, datapb.CompactionType_Level0DeleteCompaction, task.GetType())
expectedSegs := []int64{100, 101, 102}
assert.ElementsMatch(t, expectedSegs, task.GetInputSegments())
return nil
}).Return(nil)
mockAlloc.EXPECT().AllocID(mock.Anything).Return(19530, nil).Maybe()
<-triggerManager.GetPauseCompactionChan(100, 10)
defer func() {
<-triggerManager.GetResumeCompactionChan(100, 10)
}()
triggerManager.Start()
defer triggerManager.Stop()
time.Sleep(3 * time.Second)
}

View File

@ -33,6 +33,7 @@ import (
"github.com/milvus-io/milvus/pkg/proto/datapb"
"github.com/milvus-io/milvus/pkg/proto/indexpb"
"github.com/milvus-io/milvus/pkg/proto/internalpb"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
)
@ -42,12 +43,13 @@ type ImportChecker interface {
}
type importChecker struct {
meta *meta
broker broker.Broker
cluster Cluster
alloc allocator.Allocator
imeta ImportMeta
sjm StatsJobManager
meta *meta
broker broker.Broker
cluster Cluster
alloc allocator.Allocator
imeta ImportMeta
sjm StatsJobManager
l0CompactionTrigger TriggerManager
closeOnce sync.Once
closeChan chan struct{}
@ -59,15 +61,17 @@ func NewImportChecker(meta *meta,
alloc allocator.Allocator,
imeta ImportMeta,
sjm StatsJobManager,
l0CompactionTrigger TriggerManager,
) ImportChecker {
return &importChecker{
meta: meta,
broker: broker,
cluster: cluster,
alloc: alloc,
imeta: imeta,
sjm: sjm,
closeChan: make(chan struct{}),
meta: meta,
broker: broker,
cluster: cluster,
alloc: alloc,
imeta: imeta,
sjm: sjm,
l0CompactionTrigger: l0CompactionTrigger,
closeChan: make(chan struct{}),
}
}
@ -87,6 +91,14 @@ func (c *importChecker) Start() {
case <-ticker1.C:
jobs := c.imeta.GetJobBy(context.TODO())
for _, job := range jobs {
if !funcutil.SliceSetEqual[string](job.GetVchannels(), job.GetReadyVchannels()) {
// wait for all channels to send signals
log.Info("waiting for all channels to send signals",
zap.Strings("vchannels", job.GetVchannels()),
zap.Strings("readyVchannels", job.GetReadyVchannels()),
zap.Int64("jobID", job.GetJobID()))
continue
}
switch job.GetState() {
case internalpb.ImportJobState_Pending:
c.checkPendingJob(job)
@ -280,7 +292,7 @@ func (c *importChecker) checkPreImportingJob(job ImportJob) {
func (c *importChecker) checkImportingJob(job ImportJob) {
log := log.With(zap.Int64("jobID", job.GetJobID()))
tasks := c.imeta.GetTaskBy(context.TODO(), WithType(ImportTaskType), WithJob(job.GetJobID()))
tasks := c.imeta.GetTaskBy(context.TODO(), WithType(ImportTaskType), WithJob(job.GetJobID()), WithRequestSource())
for _, t := range tasks {
if t.GetState() != datapb.ImportTaskStateV2_Completed {
return
@ -380,11 +392,54 @@ func (c *importChecker) checkIndexBuildingJob(job ImportJob) {
log.Debug("waiting for import segments building index...", zap.Int64s("unindexed", unindexed))
return
}
buildIndexDuration := job.GetTR().RecordSpan()
metrics.ImportJobLatency.WithLabelValues(metrics.ImportStageBuildIndex).Observe(float64(buildIndexDuration.Milliseconds()))
log.Info("import job build index done", zap.Duration("jobTimeCost/buildIndex", buildIndexDuration))
// wait l0 segment import and block l0 compaction
log.Info("start to pause l0 segment compacting", zap.Int64("jobID", job.GetJobID()))
<-c.l0CompactionTrigger.GetPauseCompactionChan(job.GetJobID(), job.GetCollectionID())
log.Info("l0 segment compacting paused", zap.Int64("jobID", job.GetJobID()))
if c.waitL0ImortTaskDone(job) {
return
}
waitL0ImportDuration := job.GetTR().RecordSpan()
metrics.ImportJobLatency.WithLabelValues(metrics.ImportStageWaitL0Import).Observe(float64(buildIndexDuration.Milliseconds()))
log.Info("import job l0 import done", zap.Duration("jobTimeCost/l0Import", waitL0ImportDuration))
if c.updateSegmentState(job, originSegmentIDs, statsSegmentIDs) {
return
}
// all finished, update import job state to `Completed`.
completeTime := time.Now().Format("2006-01-02T15:04:05Z07:00")
err := c.imeta.UpdateJob(context.TODO(), job.GetJobID(), UpdateJobState(internalpb.ImportJobState_Completed), UpdateJobCompleteTime(completeTime))
if err != nil {
log.Warn("failed to update job state to Completed", zap.Error(err))
return
}
totalDuration := job.GetTR().ElapseSpan()
metrics.ImportJobLatency.WithLabelValues(metrics.TotalLabel).Observe(float64(totalDuration.Milliseconds()))
<-c.l0CompactionTrigger.GetResumeCompactionChan(job.GetJobID(), job.GetCollectionID())
log.Info("import job all completed", zap.Duration("jobTimeCost/total", totalDuration))
}
func (c *importChecker) waitL0ImortTaskDone(job ImportJob) bool {
// wait all lo import tasks to be completed
l0ImportTasks := c.imeta.GetTaskBy(context.TODO(), WithType(ImportTaskType), WithJob(job.GetJobID()), WithL0CompactionSource())
for _, t := range l0ImportTasks {
if t.GetState() != datapb.ImportTaskStateV2_Completed {
log.Info("waiting for l0 import task...",
zap.Int64s("taskIDs", lo.Map(l0ImportTasks, func(t ImportTask, _ int) int64 {
return t.GetTaskID()
})))
return true
}
}
return false
}
func (c *importChecker) updateSegmentState(job ImportJob, originSegmentIDs, statsSegmentIDs []int64) bool {
// Here, all segment indexes have been successfully built, try unset isImporting flag for all segments.
isImportingSegments := lo.Filter(append(originSegmentIDs, statsSegmentIDs...), func(segmentID int64, _ int) bool {
segment := c.meta.GetSegment(context.TODO(), segmentID)
@ -397,13 +452,13 @@ func (c *importChecker) checkIndexBuildingJob(job ImportJob) {
channels, err := c.meta.GetSegmentsChannels(isImportingSegments)
if err != nil {
log.Warn("get segments channels failed", zap.Error(err))
return
return true
}
for _, segmentID := range isImportingSegments {
channelCP := c.meta.GetChannelCheckpoint(channels[segmentID])
if channelCP == nil {
log.Warn("nil channel checkpoint")
return
return true
}
op1 := UpdateStartPosition([]*datapb.SegmentStartPosition{{StartPosition: channelCP, SegmentID: segmentID}})
op2 := UpdateDmlPosition(segmentID, channelCP)
@ -411,20 +466,10 @@ func (c *importChecker) checkIndexBuildingJob(job ImportJob) {
err = c.meta.UpdateSegmentsInfo(context.TODO(), op1, op2, op3)
if err != nil {
log.Warn("update import segment failed", zap.Error(err))
return
return true
}
}
// all finished, update import job state to `Completed`.
completeTime := time.Now().Format("2006-01-02T15:04:05Z07:00")
err = c.imeta.UpdateJob(context.TODO(), job.GetJobID(), UpdateJobState(internalpb.ImportJobState_Completed), UpdateJobCompleteTime(completeTime))
if err != nil {
log.Warn("failed to update job state to Completed", zap.Error(err))
return
}
totalDuration := job.GetTR().ElapseSpan()
metrics.ImportJobLatency.WithLabelValues(metrics.TotalLabel).Observe(float64(totalDuration.Milliseconds()))
log.Info("import job all completed", zap.Duration("jobTimeCost/total", totalDuration))
return false
}
func (c *importChecker) checkFailedJob(job ImportJob) {

View File

@ -23,17 +23,22 @@ import (
"time"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/internal/datacoord/allocator"
broker2 "github.com/milvus-io/milvus/internal/datacoord/broker"
"github.com/milvus-io/milvus/internal/metastore/mocks"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/proto/datapb"
"github.com/milvus-io/milvus/pkg/proto/indexpb"
"github.com/milvus-io/milvus/pkg/proto/internalpb"
"github.com/milvus-io/milvus/pkg/proto/rootcoordpb"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/timerecord"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
)
@ -74,8 +79,13 @@ func (s *ImportCheckerSuite) SetupTest() {
s.NoError(err)
sjm := NewMockStatsJobManager(s.T())
l0CompactionTrigger := NewMockTriggerManager(s.T())
compactionChan := make(chan struct{}, 1)
close(compactionChan)
l0CompactionTrigger.EXPECT().GetPauseCompactionChan(mock.Anything, mock.Anything).Return(compactionChan).Maybe()
l0CompactionTrigger.EXPECT().GetResumeCompactionChan(mock.Anything, mock.Anything).Return(compactionChan).Maybe()
checker := NewImportChecker(meta, broker, cluster, s.alloc, imeta, sjm).(*importChecker)
checker := NewImportChecker(meta, broker, cluster, s.alloc, imeta, sjm, l0CompactionTrigger).(*importChecker)
s.checker = checker
job := &importJob{
@ -508,3 +518,208 @@ func (s *ImportCheckerSuite) TestCheckCollection() {
func TestImportChecker(t *testing.T) {
suite.Run(t, new(ImportCheckerSuite))
}
func TestImportCheckerCompaction(t *testing.T) {
paramtable.Init()
Params.Save(Params.DataCoordCfg.ImportCheckIntervalHigh.Key, "1")
defer Params.Reset(Params.DataCoordCfg.ImportCheckIntervalHigh.Key)
Params.Save(Params.DataCoordCfg.ImportCheckIntervalLow.Key, "10000")
defer Params.Reset(Params.DataCoordCfg.ImportCheckIntervalLow.Key)
// prepare objects
catalog := mocks.NewDataCoordCatalog(t)
catalog.EXPECT().ListImportJobs(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListPreImportTasks(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListImportTasks(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListChannelCheckpoint(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListIndexes(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil)
cluster := NewMockCluster(t)
alloc := allocator.NewMockAllocator(t)
imeta, err := NewImportMeta(context.TODO(), catalog)
assert.NoError(t, err)
broker := broker2.NewMockBroker(t)
broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(&rootcoordpb.ShowCollectionIDsResponse{}, nil)
meta, err := newMeta(context.TODO(), catalog, nil, broker)
sjm := NewMockStatsJobManager(t)
l0CompactionTrigger := NewMockTriggerManager(t)
compactionChan := make(chan struct{}, 1)
close(compactionChan)
l0CompactionTrigger.EXPECT().GetPauseCompactionChan(mock.Anything, mock.Anything).Return(compactionChan).Maybe()
l0CompactionTrigger.EXPECT().GetResumeCompactionChan(mock.Anything, mock.Anything).Return(compactionChan).Maybe()
checker := NewImportChecker(meta, broker, cluster, alloc, imeta, sjm, l0CompactionTrigger).(*importChecker)
job := &importJob{
ImportJob: &datapb.ImportJob{
JobID: 1001,
CollectionID: 1,
PartitionIDs: []int64{2},
ReadyVchannels: []string{"ch0"},
Vchannels: []string{"ch0", "ch1"},
State: internalpb.ImportJobState_Pending,
TimeoutTs: tsoutil.ComposeTSByTime(time.Now().Add(time.Hour), 0),
CleanupTs: tsoutil.ComposeTSByTime(time.Now().Add(time.Hour), 0),
Files: []*internalpb.ImportFile{
{
Id: 1,
Paths: []string{"a.json"},
},
{
Id: 2,
Paths: []string{"b.json"},
},
{
Id: 3,
Paths: []string{"c.json"},
},
},
},
tr: timerecord.NewTimeRecorder("import job"),
}
catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once()
err = imeta.AddJob(context.TODO(), job)
assert.NoError(t, err)
jobID := job.GetJobID()
// start check
go checker.Start()
// sleep 1.5s and ready the job, go to pending stats
time.Sleep(1500 * time.Millisecond)
catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once()
job2 := &importJob{
ImportJob: &datapb.ImportJob{
JobID: 1001,
CollectionID: 1,
PartitionIDs: []int64{2},
ReadyVchannels: []string{"ch1"},
Vchannels: []string{"ch0", "ch1"},
State: internalpb.ImportJobState_Pending,
TimeoutTs: tsoutil.ComposeTSByTime(time.Now().Add(time.Hour), 0),
CleanupTs: tsoutil.ComposeTSByTime(time.Now().Add(time.Hour), 0),
Files: []*internalpb.ImportFile{
{
Id: 1,
Paths: []string{"a.json"},
},
{
Id: 2,
Paths: []string{"b.json"},
},
{
Id: 3,
Paths: []string{"c.json"},
},
},
},
tr: timerecord.NewTimeRecorder("import job"),
}
err = imeta.AddJob(context.TODO(), job2)
assert.NoError(t, err)
log.Info("job ready")
// check pending
alloc.EXPECT().AllocN(mock.Anything).RunAndReturn(func(n int64) (int64, int64, error) {
id := rand.Int63()
return id, id + n, nil
}).Maybe()
alloc.EXPECT().AllocID(mock.Anything).Return(rand.Int63(), nil).Maybe()
catalog.EXPECT().SavePreImportTask(mock.Anything, mock.Anything).Return(nil).Twice()
catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once()
assert.Eventually(t, func() bool {
job := imeta.GetJob(context.TODO(), jobID)
preimportTasks := imeta.GetTaskBy(context.TODO(), WithJob(job.GetJobID()), WithType(PreImportTaskType))
taskLen := len(preimportTasks)
log.Info("job pre-importing", zap.Any("taskLen", taskLen), zap.Any("jobState", job.GetState()))
return taskLen == 2 && job.GetState() == internalpb.ImportJobState_PreImporting
}, 2*time.Second, 500*time.Millisecond)
log.Info("job pre-importing")
// check pre-importing
catalog.EXPECT().SaveImportTask(mock.Anything, mock.Anything).Return(nil).Once()
catalog.EXPECT().SavePreImportTask(mock.Anything, mock.Anything).Return(nil).Twice()
catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once()
preimportTasks := imeta.GetTaskBy(context.TODO(), WithJob(job.GetJobID()), WithType(PreImportTaskType))
for _, pt := range preimportTasks {
err := imeta.UpdateTask(context.TODO(), pt.GetTaskID(), UpdateState(datapb.ImportTaskStateV2_Completed))
assert.NoError(t, err)
}
assert.Eventually(t, func() bool {
job := imeta.GetJob(context.TODO(), jobID)
importTasks := imeta.GetTaskBy(context.TODO(), WithJob(job.GetJobID()), WithType(ImportTaskType))
return len(importTasks) == 1 && job.GetState() == internalpb.ImportJobState_Importing
}, 2*time.Second, 100*time.Millisecond)
log.Info("job importing")
// check importing
catalog.EXPECT().AddSegment(mock.Anything, mock.Anything).Return(nil).Once()
catalog.EXPECT().AlterSegments(mock.Anything, mock.Anything).Return(nil).Once()
catalog.EXPECT().SaveChannelCheckpoint(mock.Anything, mock.Anything, mock.Anything).Return(nil).Once()
catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once()
catalog.EXPECT().SaveImportTask(mock.Anything, mock.Anything).Return(nil).Once()
importTasks := imeta.GetTaskBy(context.TODO(), WithJob(job.GetJobID()), WithType(ImportTaskType))
for _, it := range importTasks {
segment := &SegmentInfo{
SegmentInfo: &datapb.SegmentInfo{
ID: rand.Int63(),
State: commonpb.SegmentState_Flushed,
IsImporting: true,
InsertChannel: "ch0",
},
}
err := checker.meta.AddSegment(context.Background(), segment)
assert.NoError(t, err)
err = imeta.UpdateTask(context.TODO(), it.GetTaskID(), UpdateState(datapb.ImportTaskStateV2_Completed),
UpdateSegmentIDs([]int64{segment.GetID()}), UpdateStatsSegmentIDs([]int64{rand.Int63()}))
assert.NoError(t, err)
err = checker.meta.UpdateChannelCheckpoint(context.TODO(), segment.GetInsertChannel(), &msgpb.MsgPosition{MsgID: []byte{0}})
assert.NoError(t, err)
}
assert.Eventually(t, func() bool {
job := imeta.GetJob(context.TODO(), jobID)
return job.GetState() == internalpb.ImportJobState_Stats
}, 2*time.Second, 100*time.Millisecond)
log.Info("job stats")
// check stats
catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once()
sjm.EXPECT().GetStatsTask(mock.Anything, mock.Anything).Return(&indexpb.StatsTask{
State: indexpb.JobState_JobStateFinished,
}).Once()
assert.Eventually(t, func() bool {
job := imeta.GetJob(context.TODO(), jobID)
return job.GetState() == internalpb.ImportJobState_IndexBuilding
}, 2*time.Second, 100*time.Millisecond)
log.Info("job index building")
// wait l0 import task
catalog.EXPECT().SaveImportTask(mock.Anything, mock.Anything).Return(nil).Once()
imeta.AddTask(context.TODO(), &importTask{
ImportTaskV2: &datapb.ImportTaskV2{
JobID: jobID,
TaskID: 100000,
Source: datapb.ImportTaskSourceV2_L0Compaction,
State: datapb.ImportTaskStateV2_InProgress,
},
})
time.Sleep(1200 * time.Millisecond)
catalog.EXPECT().SaveImportTask(mock.Anything, mock.Anything).Return(nil).Once()
imeta.UpdateTask(context.TODO(), 100000, UpdateState(datapb.ImportTaskStateV2_Completed))
log.Info("job l0 compaction")
// check index building
catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once()
assert.Eventually(t, func() bool {
job := imeta.GetJob(context.TODO(), jobID)
return job.GetState() == internalpb.ImportJobState_Completed
}, 2*time.Second, 100*time.Millisecond)
log.Info("job completed")
}

View File

@ -104,6 +104,7 @@ type ImportJob interface {
GetCollectionName() string
GetPartitionIDs() []int64
GetVchannels() []string
GetReadyVchannels() []string
GetSchema() *schemapb.CollectionSchema
GetTimeoutTs() uint64
GetCleanupTs() uint64
@ -115,6 +116,7 @@ type ImportJob interface {
GetFiles() []*internalpb.ImportFile
GetOptions() []*commonpb.KeyValuePair
GetTR() *timerecord.TimeRecorder
GetDataTs() uint64
Clone() ImportJob
}

View File

@ -21,6 +21,7 @@ import (
"time"
"github.com/hashicorp/golang-lru/v2/expirable"
"github.com/samber/lo"
"golang.org/x/exp/maps"
"github.com/milvus-io/milvus/internal/json"
@ -140,6 +141,13 @@ func NewImportMeta(ctx context.Context, catalog metastore.DataCoordCatalog) (Imp
func (m *importMeta) AddJob(ctx context.Context, job ImportJob) error {
m.mu.Lock()
defer m.mu.Unlock()
originJob := m.jobs[job.GetJobID()]
if originJob != nil {
originJob := originJob.Clone()
internalJob := originJob.(*importJob).ImportJob
internalJob.ReadyVchannels = lo.Union(originJob.GetReadyVchannels(), job.GetReadyVchannels())
job = originJob
}
err := m.catalog.SaveImportJob(ctx, job.(*importJob).ImportJob)
if err != nil {
return err

View File

@ -90,13 +90,15 @@ func TestImportMeta_Job(t *testing.T) {
jobIDs := []int64{1000, 2000, 3000}
for i, jobID := range jobIDs {
channel := fmt.Sprintf("ch-%d", rand.Int63())
var job ImportJob = &importJob{
ImportJob: &datapb.ImportJob{
JobID: jobID,
CollectionID: rand.Int63(),
PartitionIDs: []int64{rand.Int63()},
Vchannels: []string{fmt.Sprintf("ch-%d", rand.Int63())},
State: internalpb.ImportJobState_Pending,
JobID: jobID,
CollectionID: rand.Int63(),
PartitionIDs: []int64{rand.Int63()},
Vchannels: []string{channel},
ReadyVchannels: []string{channel},
State: internalpb.ImportJobState_Pending,
},
}
err = im.AddJob(context.TODO(), job)
@ -110,7 +112,7 @@ func TestImportMeta_Job(t *testing.T) {
err = im.AddJob(context.TODO(), job)
assert.NoError(t, err)
ret = im.GetJob(context.TODO(), jobID)
assert.Equal(t, job, ret)
assert.EqualValues(t, job, ret)
jobs = im.GetJobBy(context.TODO())
assert.Equal(t, i+1, len(jobs))
}
@ -149,6 +151,48 @@ func TestImportMeta_Job(t *testing.T) {
assert.Equal(t, 2, count)
}
func TestImportMetaAddJob(t *testing.T) {
catalog := mocks.NewDataCoordCatalog(t)
catalog.EXPECT().ListImportJobs(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListPreImportTasks(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListImportTasks(mock.Anything).Return(nil, nil)
catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil)
im, err := NewImportMeta(context.TODO(), catalog)
assert.NoError(t, err)
var job ImportJob = &importJob{
ImportJob: &datapb.ImportJob{
JobID: 10000,
CollectionID: rand.Int63(),
PartitionIDs: []int64{rand.Int63()},
Vchannels: []string{"ch-1", "ch-2"},
ReadyVchannels: []string{"ch-1"},
State: internalpb.ImportJobState_Pending,
},
}
err = im.AddJob(context.TODO(), job)
assert.NoError(t, err)
job = &importJob{
ImportJob: &datapb.ImportJob{
JobID: 10000,
CollectionID: rand.Int63(),
PartitionIDs: []int64{rand.Int63()},
Vchannels: []string{"ch-1", "ch-2"},
ReadyVchannels: []string{"ch-2"},
State: internalpb.ImportJobState_Pending,
},
}
err = im.AddJob(context.TODO(), job)
assert.NoError(t, err)
job = im.GetJob(context.TODO(), 10000)
assert.NotNil(t, job)
assert.Equal(t, []string{"ch-1", "ch-2"}, job.GetVchannels())
assert.Equal(t, []string{"ch-1", "ch-2"}, job.GetReadyVchannels())
}
func TestImportMeta_ImportTask(t *testing.T) {
catalog := mocks.NewDataCoordCatalog(t)
catalog.EXPECT().ListImportJobs(mock.Anything).Return(nil, nil)

View File

@ -68,6 +68,18 @@ func WithStates(states ...datapb.ImportTaskStateV2) ImportTaskFilter {
}
}
func WithRequestSource() ImportTaskFilter {
return func(task ImportTask) bool {
return task.GetSource() == datapb.ImportTaskSourceV2_Request
}
}
func WithL0CompactionSource() ImportTaskFilter {
return func(task ImportTask) bool {
return task.GetSource() == datapb.ImportTaskSourceV2_L0Compaction
}
}
type UpdateAction func(task ImportTask)
func UpdateState(state datapb.ImportTaskStateV2) UpdateAction {
@ -150,6 +162,7 @@ type ImportTask interface {
GetTR() *timerecord.TimeRecorder
GetSlots() int64
Clone() ImportTask
GetSource() datapb.ImportTaskSourceV2
}
type preImportTask struct {
@ -176,6 +189,10 @@ func (p *preImportTask) Clone() ImportTask {
}
}
func (p *preImportTask) GetSource() datapb.ImportTaskSourceV2 {
return datapb.ImportTaskSourceV2_Request
}
func (p *preImportTask) MarshalJSON() ([]byte, error) {
importTask := metricsinfo.ImportTask{
JobID: p.GetJobID(),

View File

@ -156,7 +156,7 @@ func AssignSegments(job ImportJob, task ImportTask, alloc allocator.Allocator, m
defer cancel()
for size > 0 {
segmentInfo, err := AllocImportSegment(ctx, alloc, meta,
task.GetJobID(), task.GetTaskID(), task.GetCollectionID(), partitionID, vchannel, segmentLevel)
task.GetJobID(), task.GetTaskID(), task.GetCollectionID(), partitionID, vchannel, job.GetDataTs(), segmentLevel)
if err != nil {
return err
}
@ -183,6 +183,7 @@ func AllocImportSegment(ctx context.Context,
jobID int64, taskID int64,
collectionID UniqueID, partitionID UniqueID,
channelName string,
dataTimestamp uint64,
level datapb.SegmentLevel,
) (*SegmentInfo, error) {
log := log.Ctx(ctx)
@ -191,9 +192,12 @@ func AllocImportSegment(ctx context.Context,
log.Error("failed to alloc id for import segment", zap.Error(err))
return nil, err
}
ts, err := alloc.AllocTimestamp(ctx)
if err != nil {
return nil, err
ts := dataTimestamp
if ts == 0 {
ts, err = alloc.AllocTimestamp(ctx)
if err != nil {
return nil, err
}
}
position := &msgpb.MsgPosition{
ChannelName: channelName,
@ -263,9 +267,13 @@ func AssembleImportRequest(task ImportTask, job ImportJob, meta *meta, alloc all
}
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
ts, err := alloc.AllocTimestamp(ctx)
if err != nil {
return nil, err
ts := job.GetDataTs()
var err error
if ts == 0 {
ts, err = alloc.AllocTimestamp(ctx)
if err != nil {
return nil, err
}
}
totalRows := lo.SumBy(task.GetFileStats(), func(stat *datapb.ImportFileStats) int64 {

View File

@ -33,6 +33,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/datacoord/allocator"
"github.com/milvus-io/milvus/internal/datacoord/broker"
broker2 "github.com/milvus-io/milvus/internal/datacoord/broker"
"github.com/milvus-io/milvus/internal/json"
"github.com/milvus-io/milvus/internal/metastore/mocks"
mocks2 "github.com/milvus-io/milvus/internal/mocks"
@ -41,6 +42,7 @@ import (
"github.com/milvus-io/milvus/pkg/proto/datapb"
"github.com/milvus-io/milvus/pkg/proto/indexpb"
"github.com/milvus-io/milvus/pkg/proto/internalpb"
"github.com/milvus-io/milvus/pkg/proto/rootcoordpb"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
"github.com/milvus-io/milvus/pkg/util/paramtable"
@ -130,6 +132,64 @@ func TestImportUtil_NewImportTasks(t *testing.T) {
}
}
func TestImportUtil_NewImportTasksWithDataTt(t *testing.T) {
dataSize := paramtable.Get().DataCoordCfg.SegmentMaxSize.GetAsInt64() * 1024 * 1024
fileGroups := [][]*datapb.ImportFileStats{
{
{
ImportFile: &internalpb.ImportFile{Id: 0, Paths: []string{"a.json"}},
HashedStats: map[string]*datapb.PartitionImportStats{"c0": {PartitionDataSize: map[int64]int64{100: dataSize}}},
},
{
ImportFile: &internalpb.ImportFile{Id: 1, Paths: []string{"b.json"}},
HashedStats: map[string]*datapb.PartitionImportStats{"c0": {PartitionDataSize: map[int64]int64{100: dataSize * 2}}},
},
},
{
{
ImportFile: &internalpb.ImportFile{Id: 2, Paths: []string{"c.npy", "d.npy"}},
HashedStats: map[string]*datapb.PartitionImportStats{"c0": {PartitionDataSize: map[int64]int64{100: dataSize}}},
},
{
ImportFile: &internalpb.ImportFile{Id: 3, Paths: []string{"e.npy", "f.npy"}},
HashedStats: map[string]*datapb.PartitionImportStats{"c0": {PartitionDataSize: map[int64]int64{100: dataSize * 2}}},
},
},
}
job := &importJob{
ImportJob: &datapb.ImportJob{JobID: 1, CollectionID: 2, DataTs: 100},
}
alloc := allocator.NewMockAllocator(t)
alloc.EXPECT().AllocN(mock.Anything).RunAndReturn(func(n int64) (int64, int64, error) {
id := rand.Int63()
return id, id + n, nil
})
alloc.EXPECT().AllocID(mock.Anything).Return(rand.Int63(), nil)
catalog := mocks.NewDataCoordCatalog(t)
catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListChannelCheckpoint(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListIndexes(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return(nil, nil)
catalog.EXPECT().AddSegment(mock.Anything, mock.Anything).Return(nil)
catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil)
broker := broker2.NewMockBroker(t)
broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(&rootcoordpb.ShowCollectionIDsResponse{}, nil)
meta, err := newMeta(context.TODO(), catalog, nil, broker)
assert.NoError(t, err)
tasks, err := NewImportTasks(fileGroups, job, alloc, meta)
assert.NoError(t, err)
assert.Equal(t, 2, len(tasks))
for _, task := range tasks {
segmentIDs := task.(*importTask).GetSegmentIDs()
assert.Equal(t, 3, len(segmentIDs))
}
}
func TestImportUtil_AssembleRequest(t *testing.T) {
var job ImportJob = &importJob{
ImportJob: &datapb.ImportJob{JobID: 0, CollectionID: 1, PartitionIDs: []int64{2}, Vchannels: []string{"v0"}},
@ -198,6 +258,73 @@ func TestImportUtil_AssembleRequest(t *testing.T) {
assert.Equal(t, job.GetVchannels(), importReq.GetVchannels())
}
func TestImportUtil_AssembleRequestWithDataTt(t *testing.T) {
var job ImportJob = &importJob{
ImportJob: &datapb.ImportJob{JobID: 0, CollectionID: 1, PartitionIDs: []int64{2}, Vchannels: []string{"v0"}, DataTs: 100},
}
var pt ImportTask = &preImportTask{
PreImportTask: &datapb.PreImportTask{
JobID: 0,
TaskID: 3,
CollectionID: 1,
State: datapb.ImportTaskStateV2_Pending,
},
}
preimportReq := AssemblePreImportRequest(pt, job)
assert.Equal(t, pt.GetJobID(), preimportReq.GetJobID())
assert.Equal(t, pt.GetTaskID(), preimportReq.GetTaskID())
assert.Equal(t, pt.GetCollectionID(), preimportReq.GetCollectionID())
assert.Equal(t, job.GetPartitionIDs(), preimportReq.GetPartitionIDs())
assert.Equal(t, job.GetVchannels(), preimportReq.GetVchannels())
var task ImportTask = &importTask{
ImportTaskV2: &datapb.ImportTaskV2{
JobID: 0,
TaskID: 4,
CollectionID: 1,
SegmentIDs: []int64{5, 6},
},
}
catalog := mocks.NewDataCoordCatalog(t)
catalog.EXPECT().ListChannelCheckpoint(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListIndexes(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return(nil, nil)
catalog.EXPECT().AddSegment(mock.Anything, mock.Anything).Return(nil)
catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil)
alloc := allocator.NewMockAllocator(t)
alloc.EXPECT().AllocN(mock.Anything).RunAndReturn(func(n int64) (int64, int64, error) {
id := rand.Int63()
return id, id + n, nil
})
broker := broker2.NewMockBroker(t)
broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(&rootcoordpb.ShowCollectionIDsResponse{}, nil)
meta, err := newMeta(context.TODO(), catalog, nil, broker)
assert.NoError(t, err)
segment := &SegmentInfo{
SegmentInfo: &datapb.SegmentInfo{ID: 5, IsImporting: true},
}
err = meta.AddSegment(context.Background(), segment)
assert.NoError(t, err)
segment.ID = 6
err = meta.AddSegment(context.Background(), segment)
assert.NoError(t, err)
importReq, err := AssembleImportRequest(task, job, meta, alloc)
assert.NoError(t, err)
assert.Equal(t, task.GetJobID(), importReq.GetJobID())
assert.Equal(t, task.GetTaskID(), importReq.GetTaskID())
assert.Equal(t, task.GetCollectionID(), importReq.GetCollectionID())
assert.Equal(t, job.GetPartitionIDs(), importReq.GetPartitionIDs())
assert.Equal(t, job.GetVchannels(), importReq.GetVchannels())
}
func TestImportUtil_RegroupImportFiles(t *testing.T) {
fileNum := 4096
dataSize := paramtable.Get().DataCoordCfg.SegmentMaxSize.GetAsInt64() * 1024 * 1024

View File

@ -164,6 +164,65 @@ func (_c *MockCompactionPlanContext_getCompactionInfo_Call) RunAndReturn(run fun
return _c
}
// getCompactionTasksNum provides a mock function with given fields: filters
func (_m *MockCompactionPlanContext) getCompactionTasksNum(filters ...compactionTaskFilter) int {
_va := make([]interface{}, len(filters))
for _i := range filters {
_va[_i] = filters[_i]
}
var _ca []interface{}
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
if len(ret) == 0 {
panic("no return value specified for getCompactionTasksNum")
}
var r0 int
if rf, ok := ret.Get(0).(func(...compactionTaskFilter) int); ok {
r0 = rf(filters...)
} else {
r0 = ret.Get(0).(int)
}
return r0
}
// MockCompactionPlanContext_getCompactionTasksNum_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'getCompactionTasksNum'
type MockCompactionPlanContext_getCompactionTasksNum_Call struct {
*mock.Call
}
// getCompactionTasksNum is a helper method to define mock.On call
// - filters ...compactionTaskFilter
func (_e *MockCompactionPlanContext_Expecter) getCompactionTasksNum(filters ...interface{}) *MockCompactionPlanContext_getCompactionTasksNum_Call {
return &MockCompactionPlanContext_getCompactionTasksNum_Call{Call: _e.mock.On("getCompactionTasksNum",
append([]interface{}{}, filters...)...)}
}
func (_c *MockCompactionPlanContext_getCompactionTasksNum_Call) Run(run func(filters ...compactionTaskFilter)) *MockCompactionPlanContext_getCompactionTasksNum_Call {
_c.Call.Run(func(args mock.Arguments) {
variadicArgs := make([]compactionTaskFilter, len(args)-0)
for i, a := range args[0:] {
if a != nil {
variadicArgs[i] = a.(compactionTaskFilter)
}
}
run(variadicArgs...)
})
return _c
}
func (_c *MockCompactionPlanContext_getCompactionTasksNum_Call) Return(_a0 int) *MockCompactionPlanContext_getCompactionTasksNum_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockCompactionPlanContext_getCompactionTasksNum_Call) RunAndReturn(run func(...compactionTaskFilter) int) *MockCompactionPlanContext_getCompactionTasksNum_Call {
_c.Call.Return(run)
return _c
}
// getCompactionTasksNumBySignalID provides a mock function with given fields: signalID
func (_m *MockCompactionPlanContext) getCompactionTasksNumBySignalID(signalID int64) int {
ret := _m.Called(signalID)

View File

@ -21,6 +21,104 @@ func (_m *MockTriggerManager) EXPECT() *MockTriggerManager_Expecter {
return &MockTriggerManager_Expecter{mock: &_m.Mock}
}
// GetPauseCompactionChan provides a mock function with given fields: jobID, collectionID
func (_m *MockTriggerManager) GetPauseCompactionChan(jobID int64, collectionID int64) <-chan struct{} {
ret := _m.Called(jobID, collectionID)
if len(ret) == 0 {
panic("no return value specified for GetPauseCompactionChan")
}
var r0 <-chan struct{}
if rf, ok := ret.Get(0).(func(int64, int64) <-chan struct{}); ok {
r0 = rf(jobID, collectionID)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(<-chan struct{})
}
}
return r0
}
// MockTriggerManager_GetPauseCompactionChan_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetPauseCompactionChan'
type MockTriggerManager_GetPauseCompactionChan_Call struct {
*mock.Call
}
// GetPauseCompactionChan is a helper method to define mock.On call
// - jobID int64
// - collectionID int64
func (_e *MockTriggerManager_Expecter) GetPauseCompactionChan(jobID interface{}, collectionID interface{}) *MockTriggerManager_GetPauseCompactionChan_Call {
return &MockTriggerManager_GetPauseCompactionChan_Call{Call: _e.mock.On("GetPauseCompactionChan", jobID, collectionID)}
}
func (_c *MockTriggerManager_GetPauseCompactionChan_Call) Run(run func(jobID int64, collectionID int64)) *MockTriggerManager_GetPauseCompactionChan_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(int64), args[1].(int64))
})
return _c
}
func (_c *MockTriggerManager_GetPauseCompactionChan_Call) Return(_a0 <-chan struct{}) *MockTriggerManager_GetPauseCompactionChan_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockTriggerManager_GetPauseCompactionChan_Call) RunAndReturn(run func(int64, int64) <-chan struct{}) *MockTriggerManager_GetPauseCompactionChan_Call {
_c.Call.Return(run)
return _c
}
// GetResumeCompactionChan provides a mock function with given fields: jobID, collectionID
func (_m *MockTriggerManager) GetResumeCompactionChan(jobID int64, collectionID int64) <-chan struct{} {
ret := _m.Called(jobID, collectionID)
if len(ret) == 0 {
panic("no return value specified for GetResumeCompactionChan")
}
var r0 <-chan struct{}
if rf, ok := ret.Get(0).(func(int64, int64) <-chan struct{}); ok {
r0 = rf(jobID, collectionID)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(<-chan struct{})
}
}
return r0
}
// MockTriggerManager_GetResumeCompactionChan_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetResumeCompactionChan'
type MockTriggerManager_GetResumeCompactionChan_Call struct {
*mock.Call
}
// GetResumeCompactionChan is a helper method to define mock.On call
// - jobID int64
// - collectionID int64
func (_e *MockTriggerManager_Expecter) GetResumeCompactionChan(jobID interface{}, collectionID interface{}) *MockTriggerManager_GetResumeCompactionChan_Call {
return &MockTriggerManager_GetResumeCompactionChan_Call{Call: _e.mock.On("GetResumeCompactionChan", jobID, collectionID)}
}
func (_c *MockTriggerManager_GetResumeCompactionChan_Call) Run(run func(jobID int64, collectionID int64)) *MockTriggerManager_GetResumeCompactionChan_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(int64), args[1].(int64))
})
return _c
}
func (_c *MockTriggerManager_GetResumeCompactionChan_Call) Return(_a0 <-chan struct{}) *MockTriggerManager_GetResumeCompactionChan_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockTriggerManager_GetResumeCompactionChan_Call) RunAndReturn(run func(int64, int64) <-chan struct{}) *MockTriggerManager_GetResumeCompactionChan_Call {
_c.Call.Return(run)
return _c
}
// ManualTrigger provides a mock function with given fields: ctx, collectionID, clusteringCompaction
func (_m *MockTriggerManager) ManualTrigger(ctx context.Context, collectionID int64, clusteringCompaction bool) (int64, error) {
ret := _m.Called(ctx, collectionID, clusteringCompaction)

View File

@ -382,6 +382,10 @@ func (s *Server) initDataCoord() error {
}
log.Info("init service discovery done")
s.importMeta, err = NewImportMeta(s.ctx, s.meta.catalog)
if err != nil {
return err
}
s.initCompaction()
log.Info("init compaction done")
@ -398,12 +402,8 @@ func (s *Server) initDataCoord() error {
s.initGarbageCollection(storageCli)
s.importMeta, err = NewImportMeta(s.ctx, s.meta.catalog)
if err != nil {
return err
}
s.importScheduler = NewImportScheduler(s.meta, s.cluster, s.allocator, s.importMeta)
s.importChecker = NewImportChecker(s.meta, s.broker, s.cluster, s.allocator, s.importMeta, s.jobManager)
s.importChecker = NewImportChecker(s.meta, s.broker, s.cluster, s.allocator, s.importMeta, s.jobManager, s.compactionTriggerManager)
s.syncSegmentsScheduler = newSyncSegmentsScheduler(s.meta, s.channelManager, s.sessionManager)
@ -706,7 +706,7 @@ func (s *Server) initCompaction() {
cph := newCompactionPlanHandler(s.cluster, s.sessionManager, s.meta, s.allocator, s.handler)
cph.loadMeta()
s.compactionHandler = cph
s.compactionTriggerManager = NewCompactionTriggerManager(s.allocator, s.handler, s.compactionHandler, s.meta)
s.compactionTriggerManager = NewCompactionTriggerManager(s.allocator, s.handler, s.compactionHandler, s.meta, s.importMeta)
s.compactionTrigger = newCompactionTrigger(s.meta, s.compactionHandler, s.allocator, s.handler, s.indexEngineVersionManager)
}

View File

@ -1723,16 +1723,20 @@ func (s *Server) ImportV2(ctx context.Context, in *internalpb.ImportRequestInter
log.Info("list binlogs prefixes for import", zap.Any("binlog_prefixes", files))
}
// The import task does not need to be controlled for the time being, and additional development is required later.
// Here is a comment, because the current importv2 communicates through messages and needs to ensure idempotence.
// Adding this part of the logic will cause importv2 to retry infinitely until the previous import task is completed.
// Check if the number of jobs exceeds the limit.
maxNum := paramtable.Get().DataCoordCfg.MaxImportJobNum.GetAsInt()
executingNum := s.importMeta.CountJobBy(ctx, WithoutJobStates(internalpb.ImportJobState_Completed, internalpb.ImportJobState_Failed))
if executingNum >= maxNum {
resp.Status = merr.Status(merr.WrapErrImportFailed(
fmt.Sprintf("The number of jobs has reached the limit, please try again later. " +
"If your request is set to only import a single file, " +
"please consider importing multiple files in one request for better efficiency.")))
return resp, nil
}
// maxNum := paramtable.Get().DataCoordCfg.MaxImportJobNum.GetAsInt()
// executingNum := s.importMeta.CountJobBy(ctx, WithoutJobStates(internalpb.ImportJobState_Completed, internalpb.ImportJobState_Failed))
// if executingNum >= maxNum {
// resp.Status = merr.Status(merr.WrapErrImportFailed(
// fmt.Sprintf("The number of jobs has reached the limit, please try again later. " +
// "If your request is set to only import a single file, " +
// "please consider importing multiple files in one request for better efficiency.")))
// return resp, nil
// }
// Allocate file ids.
idStart, _, err := s.allocator.AllocN(int64(len(files)) + 1)
@ -1744,15 +1748,28 @@ func (s *Server) ImportV2(ctx context.Context, in *internalpb.ImportRequestInter
importFile.Id = idStart + int64(i) + 1
return importFile
})
importCollectionInfo, err := s.handler.GetCollection(ctx, in.GetCollectionID())
if err != nil {
resp.Status = merr.Status(merr.WrapErrImportFailed(fmt.Sprint("get collection failed, err=%w", err)))
return resp, nil
}
if importCollectionInfo == nil {
resp.Status = merr.Status(merr.WrapErrCollectionNotFound(in.GetCollectionID()))
return resp, nil
}
jobID := in.GetJobID()
if jobID == 0 {
jobID = idStart
}
startTime := time.Now()
job := &importJob{
ImportJob: &datapb.ImportJob{
JobID: idStart,
JobID: jobID,
CollectionID: in.GetCollectionID(),
CollectionName: in.GetCollectionName(),
PartitionIDs: in.GetPartitionIDs(),
Vchannels: in.GetChannelNames(),
Vchannels: importCollectionInfo.VChannelNames,
Schema: in.GetSchema(),
TimeoutTs: timeoutTs,
CleanupTs: math.MaxUint64,
@ -1760,6 +1777,8 @@ func (s *Server) ImportV2(ctx context.Context, in *internalpb.ImportRequestInter
Files: files,
Options: in.GetOptions(),
StartTime: startTime.Format("2006-01-02T15:04:05Z07:00"),
ReadyVchannels: in.GetChannelNames(),
DataTs: in.GetDataTimestamp(),
},
tr: timerecord.NewTimeRecorder("import job"),
}
@ -1770,7 +1789,11 @@ func (s *Server) ImportV2(ctx context.Context, in *internalpb.ImportRequestInter
}
resp.JobID = fmt.Sprint(job.GetJobID())
log.Info("add import job done", zap.Int64("jobID", job.GetJobID()), zap.Any("files", files))
log.Info("add import job done",
zap.Int64("jobID", job.GetJobID()),
zap.Any("files", files),
zap.Strings("readyChannels", in.GetChannelNames()),
)
return resp, nil
}
@ -1790,6 +1813,7 @@ func (s *Server) GetImportProgress(ctx context.Context, in *internalpb.GetImport
resp.Status = merr.Status(merr.WrapErrImportFailed(fmt.Sprint("parse job id failed, err=%w", err)))
return resp, nil
}
job := s.importMeta.GetJob(ctx, jobID)
if job == nil {
resp.Status = merr.Status(merr.WrapErrImportFailed(fmt.Sprintf("import job does not exist, jobID=%d", jobID)))

View File

@ -22,9 +22,11 @@ import (
"github.com/milvus-io/milvus/internal/datacoord/allocator"
"github.com/milvus-io/milvus/internal/datacoord/broker"
"github.com/milvus-io/milvus/internal/datacoord/session"
"github.com/milvus-io/milvus/internal/distributed/streaming"
"github.com/milvus-io/milvus/internal/metastore/mocks"
"github.com/milvus-io/milvus/internal/metastore/model"
mocks2 "github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/mocks/distributed/mock_streaming"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/mq/msgstream"
@ -1313,6 +1315,12 @@ func TestImportV2(t *testing.T) {
assert.NoError(t, err)
assert.NotEqual(t, int32(0), resp.GetStatus().GetCode())
s.stateCode.Store(commonpb.StateCode_Healthy)
mockHandler := NewNMockHandler(t)
mockHandler.EXPECT().GetCollection(mock.Anything, mock.Anything).Return(&collectionInfo{
ID: 1000,
VChannelNames: []string{"foo_1v1"},
}, nil).Maybe()
s.handler = mockHandler
// parse timeout failed
resp, err = s.ImportV2(ctx, &internalpb.ImportRequestInternal{
@ -1397,6 +1405,7 @@ func TestImportV2(t *testing.T) {
Paths: []string{"a.json"},
},
},
ChannelNames: []string{"foo_1v1"},
})
assert.NoError(t, err)
assert.Equal(t, int32(0), resp.GetStatus().GetCode())
@ -1404,11 +1413,11 @@ func TestImportV2(t *testing.T) {
assert.Equal(t, 1, len(jobs))
// number of jobs reached the limit
Params.Save(paramtable.Get().DataCoordCfg.MaxImportJobNum.Key, "1")
resp, err = s.ImportV2(ctx, &internalpb.ImportRequestInternal{})
assert.NoError(t, err)
assert.True(t, errors.Is(merr.Error(resp.GetStatus()), merr.ErrImportFailed))
Params.Reset(paramtable.Get().DataCoordCfg.MaxImportJobNum.Key)
// Params.Save(paramtable.Get().DataCoordCfg.MaxImportJobNum.Key, "1")
// resp, err = s.ImportV2(ctx, &internalpb.ImportRequestInternal{})
// assert.NoError(t, err)
// assert.True(t, errors.Is(merr.Error(resp.GetStatus()), merr.ErrImportFailed))
// Params.Reset(paramtable.Get().DataCoordCfg.MaxImportJobNum.Key)
})
t.Run("GetImportProgress", func(t *testing.T) {
@ -1433,6 +1442,13 @@ func TestImportV2(t *testing.T) {
catalog.EXPECT().ListPreImportTasks(mock.Anything).Return(nil, nil)
catalog.EXPECT().ListImportTasks(mock.Anything).Return(nil, nil)
catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil)
wal := mock_streaming.NewMockWALAccesser(t)
b := mock_streaming.NewMockBroadcast(t)
wal.EXPECT().Broadcast().Return(b).Maybe()
b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe()
streaming.SetWALForTest(wal)
defer streaming.RecoverWALForTest()
s.importMeta, err = NewImportMeta(context.TODO(), catalog)
assert.NoError(t, err)
resp, err = s.GetImportProgress(ctx, &internalpb.GetImportProgressRequest{

View File

@ -40,6 +40,7 @@ import (
"github.com/milvus-io/milvus/internal/datanode/channel"
"github.com/milvus-io/milvus/internal/datanode/compaction"
"github.com/milvus-io/milvus/internal/datanode/importv2"
"github.com/milvus-io/milvus/internal/datanode/msghandlerimpl"
"github.com/milvus-io/milvus/internal/datanode/util"
"github.com/milvus-io/milvus/internal/flushcommon/broker"
"github.com/milvus-io/milvus/internal/flushcommon/pipeline"
@ -470,5 +471,6 @@ func getPipelineParams(node *DataNode) *util2.PipelineParams {
WriteBufferManager: node.writeBufferManager,
CheckpointUpdater: node.channelCheckpointUpdater,
Allocator: node.allocator,
MsgHandler: msghandlerimpl.NewMsgHandlerImpl(node.broker),
}
}

View File

@ -0,0 +1,96 @@
/*
* 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 msghandlerimpl
import (
"context"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/internal/distributed/streaming"
"github.com/milvus-io/milvus/internal/flushcommon/broker"
"github.com/milvus-io/milvus/internal/flushcommon/util"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/proto/internalpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/retry"
)
type msgHandlerImpl struct {
broker broker.Broker
}
func (m *msgHandlerImpl) HandleCreateSegment(ctx context.Context, vchannel string, createSegmentMsg message.ImmutableCreateSegmentMessageV2) error {
panic("unreachable code")
}
func (m *msgHandlerImpl) HandleFlush(vchannel string, flushMsg message.ImmutableFlushMessageV2) error {
panic("unreachable code")
}
func (m *msgHandlerImpl) HandleManualFlush(vchannel string, flushMsg message.ImmutableManualFlushMessageV2) error {
panic("unreachable code")
}
func (m *msgHandlerImpl) HandleImport(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg) error {
return retry.Do(ctx, func() (err error) {
defer func() {
if err == nil {
err = streaming.WAL().Broadcast().Ack(ctx, types.BroadcastAckRequest{
BroadcastID: uint64(importMsg.GetJobID()),
VChannel: vchannel,
})
}
}()
importResp, err := m.broker.ImportV2(ctx, &internalpb.ImportRequestInternal{
CollectionID: importMsg.GetCollectionID(),
CollectionName: importMsg.GetCollectionName(),
PartitionIDs: importMsg.GetPartitionIDs(),
ChannelNames: []string{vchannel},
Schema: importMsg.GetSchema(),
Files: lo.Map(importMsg.GetFiles(), util.ConvertInternalImportFile),
Options: funcutil.Map2KeyValuePair(importMsg.GetOptions()),
DataTimestamp: importMsg.GetBase().GetTimestamp(),
JobID: importMsg.GetJobID(),
})
err = merr.CheckRPCCall(importResp, err)
if errors.Is(err, merr.ErrCollectionNotFound) {
log.Ctx(ctx).Warn("import message failed because of collection not found, skip it", zap.String("job_id", importResp.GetJobID()), zap.Error(err))
return nil
}
if err != nil {
log.Ctx(ctx).Warn("import message failed", zap.String("job_id", importResp.GetJobID()), zap.Error(err))
return err
}
log.Ctx(ctx).Info("import message handled", zap.String("job_id", importResp.GetJobID()))
return nil
}, retry.AttemptAlways())
}
func NewMsgHandlerImpl(broker broker.Broker) *msgHandlerImpl {
return &msgHandlerImpl{
broker: broker,
}
}

View File

@ -0,0 +1,61 @@
/*
* 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 msghandlerimpl
import (
"context"
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus/internal/distributed/streaming"
"github.com/milvus-io/milvus/internal/flushcommon/broker"
"github.com/milvus-io/milvus/internal/mocks/distributed/mock_streaming"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)
func TestMsgHandlerImpl(t *testing.T) {
paramtable.Init()
ctx := context.Background()
b := broker.NewMockBroker(t)
m := NewMsgHandlerImpl(b)
assert.Panics(t, func() {
m.HandleCreateSegment(nil, "", nil)
})
assert.Panics(t, func() {
m.HandleFlush("", nil)
})
assert.Panics(t, func() {
m.HandleManualFlush("", nil)
})
t.Run("HandleImport success", func(t *testing.T) {
wal := mock_streaming.NewMockWALAccesser(t)
bo := mock_streaming.NewMockBroadcast(t)
wal.EXPECT().Broadcast().Return(bo)
bo.EXPECT().Ack(mock.Anything, mock.Anything).Return(nil)
streaming.SetWALForTest(wal)
defer streaming.RecoverWALForTest()
b.EXPECT().ImportV2(mock.Anything, mock.Anything).Return(nil, assert.AnError).Once()
b.EXPECT().ImportV2(mock.Anything, mock.Anything).Return(nil, nil).Once()
err := m.HandleImport(ctx, "", nil)
assert.NoError(t, err)
})
}

View File

@ -18,7 +18,14 @@
package streaming
import kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv"
// SetWALForTest initializes the singleton of wal for test.
func SetWALForTest(w WALAccesser) {
singleton = w
}
func RecoverWALForTest() {
c, _ := kvfactory.GetEtcdAndPath()
singleton = newWALAccesser(c)
}

View File

@ -6,6 +6,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/pkg/proto/datapb"
"github.com/milvus-io/milvus/pkg/proto/internalpb"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
@ -38,4 +39,5 @@ type DataCoord interface {
SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) error
DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest) (*datapb.DropVirtualChannelResponse, error)
UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) error
ImportV2(ctx context.Context, in *internalpb.ImportRequestInternal) (*internalpb.ImportResponse, error)
}

View File

@ -14,6 +14,7 @@ import (
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/proto/datapb"
"github.com/milvus-io/milvus/pkg/proto/internalpb"
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"
@ -166,3 +167,13 @@ func (dc *dataCoordBroker) UpdateSegmentStatistics(ctx context.Context, req *dat
return nil
}
func (dc *dataCoordBroker) ImportV2(ctx context.Context, in *internalpb.ImportRequestInternal) (*internalpb.ImportResponse, error) {
resp, err := dc.client.ImportV2(ctx, in)
if err := merr.CheckRPCCall(resp, err); err != nil {
log.Ctx(ctx).Warn("failed to ImportV2", zap.Error(err))
return resp, err
}
return resp, nil
}

View File

@ -15,6 +15,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/pkg/proto/datapb"
"github.com/milvus-io/milvus/pkg/proto/internalpb"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
@ -328,6 +329,28 @@ func (s *dataCoordSuite) TestUpdateSegmentStatistics() {
})
}
func (s *dataCoordSuite) TestImportV2() {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
req := &internalpb.ImportRequestInternal{}
s.Run("normal_case", func() {
s.dc.EXPECT().ImportV2(mock.Anything, mock.Anything).
Return(&internalpb.ImportResponse{Status: merr.Status(nil), JobID: "1000"}, nil)
resp, err := s.broker.ImportV2(ctx, req)
s.NoError(err)
s.Equal("1000", resp.GetJobID())
s.resetMock()
})
s.Run("datacoord_return_error", func() {
s.dc.EXPECT().ImportV2(mock.Anything, mock.Anything).
Return(nil, errors.New("mock"))
_, err := s.broker.ImportV2(ctx, req)
s.Error(err)
s.resetMock()
})
}
func TestDataCoordBroker(t *testing.T) {
suite.Run(t, new(dataCoordSuite))
}

View File

@ -6,6 +6,8 @@ import (
context "context"
datapb "github.com/milvus-io/milvus/pkg/proto/datapb"
internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb"
mock "github.com/stretchr/testify/mock"
msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
@ -215,6 +217,65 @@ func (_c *MockBroker_GetSegmentInfo_Call) RunAndReturn(run func(context.Context,
return _c
}
// ImportV2 provides a mock function with given fields: ctx, in
func (_m *MockBroker) ImportV2(ctx context.Context, in *internalpb.ImportRequestInternal) (*internalpb.ImportResponse, error) {
ret := _m.Called(ctx, in)
if len(ret) == 0 {
panic("no return value specified for ImportV2")
}
var r0 *internalpb.ImportResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.ImportRequestInternal) (*internalpb.ImportResponse, error)); ok {
return rf(ctx, in)
}
if rf, ok := ret.Get(0).(func(context.Context, *internalpb.ImportRequestInternal) *internalpb.ImportResponse); ok {
r0 = rf(ctx, in)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*internalpb.ImportResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *internalpb.ImportRequestInternal) error); ok {
r1 = rf(ctx, in)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockBroker_ImportV2_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ImportV2'
type MockBroker_ImportV2_Call struct {
*mock.Call
}
// ImportV2 is a helper method to define mock.On call
// - ctx context.Context
// - in *internalpb.ImportRequestInternal
func (_e *MockBroker_Expecter) ImportV2(ctx interface{}, in interface{}) *MockBroker_ImportV2_Call {
return &MockBroker_ImportV2_Call{Call: _e.mock.On("ImportV2", ctx, in)}
}
func (_c *MockBroker_ImportV2_Call) Run(run func(ctx context.Context, in *internalpb.ImportRequestInternal)) *MockBroker_ImportV2_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*internalpb.ImportRequestInternal))
})
return _c
}
func (_c *MockBroker_ImportV2_Call) Return(_a0 *internalpb.ImportResponse, _a1 error) *MockBroker_ImportV2_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockBroker_ImportV2_Call) RunAndReturn(run func(context.Context, *internalpb.ImportRequestInternal) (*internalpb.ImportResponse, error)) *MockBroker_ImportV2_Call {
_c.Call.Return(run)
return _c
}
// ReportTimeTick provides a mock function with given fields: ctx, msgs
func (_m *MockBroker) ReportTimeTick(ctx context.Context, msgs []*msgpb.DataNodeTtMsg) error {
ret := _m.Called(ctx, msgs)

View File

@ -30,7 +30,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/internal/datanode/compaction"
"github.com/milvus-io/milvus/internal/flushcommon/util"
"github.com/milvus-io/milvus/internal/streamingnode/server/flusher"
"github.com/milvus-io/milvus/internal/util/flowgraph"
"github.com/milvus-io/milvus/internal/util/streamingutil"
"github.com/milvus-io/milvus/pkg/log"
@ -71,7 +70,7 @@ type ddNode struct {
dropMode atomic.Value
compactionExecutor compaction.Executor
msgHandler flusher.MsgHandler
msgHandler util.MsgHandler
// for recovery
growingSegInfo map[typeutil.UniqueID]*datapb.SegmentInfo // segmentID
@ -277,6 +276,21 @@ func (ddn *ddNode) Operate(in []Msg) []Msg {
} else {
logger.Info("handle manual flush message success")
}
case commonpb.MsgType_Import:
importMsg := msg.(*msgstream.ImportMsg)
if importMsg.GetCollectionID() != ddn.collectionID {
continue
}
logger := log.With(
zap.String("vchannel", ddn.Name()),
zap.Int32("msgType", int32(msg.Type())),
)
logger.Info("receive import message")
if err := ddn.msgHandler.HandleImport(context.Background(), ddn.vChannelName, importMsg.ImportMsg); err != nil {
logger.Warn("handle import message failed", zap.Error(err))
} else {
logger.Info("handle import message success")
}
}
}
@ -332,7 +346,7 @@ func (ddn *ddNode) Close() {
}
func newDDNode(ctx context.Context, collID typeutil.UniqueID, vChannelName string, droppedSegmentIDs []typeutil.UniqueID,
sealedSegments []*datapb.SegmentInfo, growingSegments []*datapb.SegmentInfo, executor compaction.Executor, handler flusher.MsgHandler,
sealedSegments []*datapb.SegmentInfo, growingSegments []*datapb.SegmentInfo, executor compaction.Executor, handler util.MsgHandler,
) *ddNode {
baseNode := BaseNode{}
baseNode.SetMaxQueueLength(paramtable.Get().DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32())

View File

@ -28,7 +28,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/internal/datanode/compaction"
"github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_flusher"
"github.com/milvus-io/milvus/internal/mocks/flushcommon/mock_util"
"github.com/milvus-io/milvus/internal/util/flowgraph"
"github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message"
"github.com/milvus-io/milvus/pkg/mq/msgstream"
@ -97,7 +97,7 @@ func TestFlowGraph_DDNode_newDDNode(t *testing.T) {
}
func TestFlowGraph_DDNode_OperateFlush(t *testing.T) {
h := mock_flusher.NewMockFlushMsgHandler(t)
h := mock_util.NewMockMsgHandler(t)
h.EXPECT().HandleCreateSegment(mock.Anything, mock.Anything, mock.Anything).Return(nil)
h.EXPECT().HandleFlush(mock.Anything, mock.Anything).Return(nil)
h.EXPECT().HandleManualFlush(mock.Anything, mock.Anything).Return(nil)

View File

@ -14,11 +14,13 @@
// See the License for the specific language governing permissions and
// limitations under the License.
package flusher
package util
import (
"context"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/pkg/proto/internalpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
)
@ -28,4 +30,13 @@ type MsgHandler interface {
HandleFlush(vchannel string, flushMsg message.ImmutableFlushMessageV2) error
HandleManualFlush(vchannel string, flushMsg message.ImmutableManualFlushMessageV2) error
HandleImport(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg) error
}
func ConvertInternalImportFile(file *msgpb.ImportFile, _ int) *internalpb.ImportFile {
return &internalpb.ImportFile{
Id: file.GetId(),
Paths: file.GetPaths(),
}
}

View File

@ -28,7 +28,6 @@ import (
"github.com/milvus-io/milvus/internal/flushcommon/syncmgr"
"github.com/milvus-io/milvus/internal/flushcommon/writebuffer"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/streamingnode/server/flusher"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
@ -49,7 +48,7 @@ type PipelineParams struct {
WriteBufferManager writebuffer.BufferManager
CheckpointUpdater *ChannelCheckpointUpdater
Allocator allocator.Interface
MsgHandler flusher.MsgHandler
MsgHandler MsgHandler
}
// TimeRange is a range of timestamp contains the min-timestamp and max-timestamp

View File

@ -0,0 +1,239 @@
// Code generated by mockery v2.46.0. DO NOT EDIT.
package mock_streaming
import (
context "context"
message "github.com/milvus-io/milvus/pkg/streaming/util/message"
mock "github.com/stretchr/testify/mock"
types "github.com/milvus-io/milvus/pkg/streaming/util/types"
)
// MockBroadcast is an autogenerated mock type for the Broadcast type
type MockBroadcast struct {
mock.Mock
}
type MockBroadcast_Expecter struct {
mock *mock.Mock
}
func (_m *MockBroadcast) EXPECT() *MockBroadcast_Expecter {
return &MockBroadcast_Expecter{mock: &_m.Mock}
}
// Ack provides a mock function with given fields: ctx, req
func (_m *MockBroadcast) Ack(ctx context.Context, req types.BroadcastAckRequest) error {
ret := _m.Called(ctx, req)
if len(ret) == 0 {
panic("no return value specified for Ack")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, types.BroadcastAckRequest) error); ok {
r0 = rf(ctx, req)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockBroadcast_Ack_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Ack'
type MockBroadcast_Ack_Call struct {
*mock.Call
}
// Ack is a helper method to define mock.On call
// - ctx context.Context
// - req types.BroadcastAckRequest
func (_e *MockBroadcast_Expecter) Ack(ctx interface{}, req interface{}) *MockBroadcast_Ack_Call {
return &MockBroadcast_Ack_Call{Call: _e.mock.On("Ack", ctx, req)}
}
func (_c *MockBroadcast_Ack_Call) Run(run func(ctx context.Context, req types.BroadcastAckRequest)) *MockBroadcast_Ack_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(types.BroadcastAckRequest))
})
return _c
}
func (_c *MockBroadcast_Ack_Call) Return(_a0 error) *MockBroadcast_Ack_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockBroadcast_Ack_Call) RunAndReturn(run func(context.Context, types.BroadcastAckRequest) error) *MockBroadcast_Ack_Call {
_c.Call.Return(run)
return _c
}
// Append provides a mock function with given fields: ctx, msg
func (_m *MockBroadcast) Append(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) {
ret := _m.Called(ctx, msg)
if len(ret) == 0 {
panic("no return value specified for Append")
}
var r0 *types.BroadcastAppendResult
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)); ok {
return rf(ctx, msg)
}
if rf, ok := ret.Get(0).(func(context.Context, message.BroadcastMutableMessage) *types.BroadcastAppendResult); ok {
r0 = rf(ctx, msg)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*types.BroadcastAppendResult)
}
}
if rf, ok := ret.Get(1).(func(context.Context, message.BroadcastMutableMessage) error); ok {
r1 = rf(ctx, msg)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockBroadcast_Append_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Append'
type MockBroadcast_Append_Call struct {
*mock.Call
}
// Append is a helper method to define mock.On call
// - ctx context.Context
// - msg message.BroadcastMutableMessage
func (_e *MockBroadcast_Expecter) Append(ctx interface{}, msg interface{}) *MockBroadcast_Append_Call {
return &MockBroadcast_Append_Call{Call: _e.mock.On("Append", ctx, msg)}
}
func (_c *MockBroadcast_Append_Call) Run(run func(ctx context.Context, msg message.BroadcastMutableMessage)) *MockBroadcast_Append_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(message.BroadcastMutableMessage))
})
return _c
}
func (_c *MockBroadcast_Append_Call) Return(_a0 *types.BroadcastAppendResult, _a1 error) *MockBroadcast_Append_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockBroadcast_Append_Call) RunAndReturn(run func(context.Context, message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)) *MockBroadcast_Append_Call {
_c.Call.Return(run)
return _c
}
// BlockUntilResourceKeyAckAll provides a mock function with given fields: ctx, rk
func (_m *MockBroadcast) BlockUntilResourceKeyAckAll(ctx context.Context, rk message.ResourceKey) error {
ret := _m.Called(ctx, rk)
if len(ret) == 0 {
panic("no return value specified for BlockUntilResourceKeyAckAll")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, message.ResourceKey) error); ok {
r0 = rf(ctx, rk)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockBroadcast_BlockUntilResourceKeyAckAll_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BlockUntilResourceKeyAckAll'
type MockBroadcast_BlockUntilResourceKeyAckAll_Call struct {
*mock.Call
}
// BlockUntilResourceKeyAckAll is a helper method to define mock.On call
// - ctx context.Context
// - rk message.ResourceKey
func (_e *MockBroadcast_Expecter) BlockUntilResourceKeyAckAll(ctx interface{}, rk interface{}) *MockBroadcast_BlockUntilResourceKeyAckAll_Call {
return &MockBroadcast_BlockUntilResourceKeyAckAll_Call{Call: _e.mock.On("BlockUntilResourceKeyAckAll", ctx, rk)}
}
func (_c *MockBroadcast_BlockUntilResourceKeyAckAll_Call) Run(run func(ctx context.Context, rk message.ResourceKey)) *MockBroadcast_BlockUntilResourceKeyAckAll_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(message.ResourceKey))
})
return _c
}
func (_c *MockBroadcast_BlockUntilResourceKeyAckAll_Call) Return(_a0 error) *MockBroadcast_BlockUntilResourceKeyAckAll_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockBroadcast_BlockUntilResourceKeyAckAll_Call) RunAndReturn(run func(context.Context, message.ResourceKey) error) *MockBroadcast_BlockUntilResourceKeyAckAll_Call {
_c.Call.Return(run)
return _c
}
// BlockUntilResourceKeyAckOnce provides a mock function with given fields: ctx, rk
func (_m *MockBroadcast) BlockUntilResourceKeyAckOnce(ctx context.Context, rk message.ResourceKey) error {
ret := _m.Called(ctx, rk)
if len(ret) == 0 {
panic("no return value specified for BlockUntilResourceKeyAckOnce")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, message.ResourceKey) error); ok {
r0 = rf(ctx, rk)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockBroadcast_BlockUntilResourceKeyAckOnce_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BlockUntilResourceKeyAckOnce'
type MockBroadcast_BlockUntilResourceKeyAckOnce_Call struct {
*mock.Call
}
// BlockUntilResourceKeyAckOnce is a helper method to define mock.On call
// - ctx context.Context
// - rk message.ResourceKey
func (_e *MockBroadcast_Expecter) BlockUntilResourceKeyAckOnce(ctx interface{}, rk interface{}) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call {
return &MockBroadcast_BlockUntilResourceKeyAckOnce_Call{Call: _e.mock.On("BlockUntilResourceKeyAckOnce", ctx, rk)}
}
func (_c *MockBroadcast_BlockUntilResourceKeyAckOnce_Call) Run(run func(ctx context.Context, rk message.ResourceKey)) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(message.ResourceKey))
})
return _c
}
func (_c *MockBroadcast_BlockUntilResourceKeyAckOnce_Call) Return(_a0 error) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockBroadcast_BlockUntilResourceKeyAckOnce_Call) RunAndReturn(run func(context.Context, message.ResourceKey) error) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call {
_c.Call.Return(run)
return _c
}
// NewMockBroadcast creates a new instance of MockBroadcast. 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 NewMockBroadcast(t interface {
mock.TestingT
Cleanup(func())
}) *MockBroadcast {
mock := &MockBroadcast{}
mock.Mock.Test(t)
t.Cleanup(func() { mock.AssertExpectations(t) })
return mock
}

View File

@ -0,0 +1,230 @@
// Code generated by mockery v2.46.0. DO NOT EDIT.
package mock_util
import (
context "context"
message "github.com/milvus-io/milvus/pkg/streaming/util/message"
mock "github.com/stretchr/testify/mock"
msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
)
// MockMsgHandler is an autogenerated mock type for the MsgHandler type
type MockMsgHandler struct {
mock.Mock
}
type MockMsgHandler_Expecter struct {
mock *mock.Mock
}
func (_m *MockMsgHandler) EXPECT() *MockMsgHandler_Expecter {
return &MockMsgHandler_Expecter{mock: &_m.Mock}
}
// HandleCreateSegment provides a mock function with given fields: ctx, vchannel, createSegmentMsg
func (_m *MockMsgHandler) HandleCreateSegment(ctx context.Context, vchannel string, createSegmentMsg message.ImmutableCreateSegmentMessageV2) error {
ret := _m.Called(ctx, vchannel, createSegmentMsg)
if len(ret) == 0 {
panic("no return value specified for HandleCreateSegment")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, string, message.ImmutableCreateSegmentMessageV2) error); ok {
r0 = rf(ctx, vchannel, createSegmentMsg)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockMsgHandler_HandleCreateSegment_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleCreateSegment'
type MockMsgHandler_HandleCreateSegment_Call struct {
*mock.Call
}
// HandleCreateSegment is a helper method to define mock.On call
// - ctx context.Context
// - vchannel string
// - createSegmentMsg message.specializedImmutableMessage[*messagespb.CreateSegmentMessageHeader,*messagespb.CreateSegmentMessageBody]
func (_e *MockMsgHandler_Expecter) HandleCreateSegment(ctx interface{}, vchannel interface{}, createSegmentMsg interface{}) *MockMsgHandler_HandleCreateSegment_Call {
return &MockMsgHandler_HandleCreateSegment_Call{Call: _e.mock.On("HandleCreateSegment", ctx, vchannel, createSegmentMsg)}
}
func (_c *MockMsgHandler_HandleCreateSegment_Call) Run(run func(ctx context.Context, vchannel string, createSegmentMsg message.ImmutableCreateSegmentMessageV2)) *MockMsgHandler_HandleCreateSegment_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(string), args[2].(message.ImmutableCreateSegmentMessageV2))
})
return _c
}
func (_c *MockMsgHandler_HandleCreateSegment_Call) Return(_a0 error) *MockMsgHandler_HandleCreateSegment_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockMsgHandler_HandleCreateSegment_Call) RunAndReturn(run func(context.Context, string, message.ImmutableCreateSegmentMessageV2) error) *MockMsgHandler_HandleCreateSegment_Call {
_c.Call.Return(run)
return _c
}
// HandleFlush provides a mock function with given fields: vchannel, flushMsg
func (_m *MockMsgHandler) HandleFlush(vchannel string, flushMsg message.ImmutableFlushMessageV2) error {
ret := _m.Called(vchannel, flushMsg)
if len(ret) == 0 {
panic("no return value specified for HandleFlush")
}
var r0 error
if rf, ok := ret.Get(0).(func(string, message.ImmutableFlushMessageV2) error); ok {
r0 = rf(vchannel, flushMsg)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockMsgHandler_HandleFlush_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleFlush'
type MockMsgHandler_HandleFlush_Call struct {
*mock.Call
}
// HandleFlush is a helper method to define mock.On call
// - vchannel string
// - flushMsg message.specializedImmutableMessage[*messagespb.FlushMessageHeader,*messagespb.FlushMessageBody]
func (_e *MockMsgHandler_Expecter) HandleFlush(vchannel interface{}, flushMsg interface{}) *MockMsgHandler_HandleFlush_Call {
return &MockMsgHandler_HandleFlush_Call{Call: _e.mock.On("HandleFlush", vchannel, flushMsg)}
}
func (_c *MockMsgHandler_HandleFlush_Call) Run(run func(vchannel string, flushMsg message.ImmutableFlushMessageV2)) *MockMsgHandler_HandleFlush_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(string), args[1].(message.ImmutableFlushMessageV2))
})
return _c
}
func (_c *MockMsgHandler_HandleFlush_Call) Return(_a0 error) *MockMsgHandler_HandleFlush_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockMsgHandler_HandleFlush_Call) RunAndReturn(run func(string, message.ImmutableFlushMessageV2) error) *MockMsgHandler_HandleFlush_Call {
_c.Call.Return(run)
return _c
}
// HandleImport provides a mock function with given fields: ctx, vchannel, importMsg
func (_m *MockMsgHandler) HandleImport(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg) error {
ret := _m.Called(ctx, vchannel, importMsg)
if len(ret) == 0 {
panic("no return value specified for HandleImport")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, string, *msgpb.ImportMsg) error); ok {
r0 = rf(ctx, vchannel, importMsg)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockMsgHandler_HandleImport_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleImport'
type MockMsgHandler_HandleImport_Call struct {
*mock.Call
}
// HandleImport is a helper method to define mock.On call
// - ctx context.Context
// - vchannel string
// - importMsg *msgpb.ImportMsg
func (_e *MockMsgHandler_Expecter) HandleImport(ctx interface{}, vchannel interface{}, importMsg interface{}) *MockMsgHandler_HandleImport_Call {
return &MockMsgHandler_HandleImport_Call{Call: _e.mock.On("HandleImport", ctx, vchannel, importMsg)}
}
func (_c *MockMsgHandler_HandleImport_Call) Run(run func(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg)) *MockMsgHandler_HandleImport_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(string), args[2].(*msgpb.ImportMsg))
})
return _c
}
func (_c *MockMsgHandler_HandleImport_Call) Return(_a0 error) *MockMsgHandler_HandleImport_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockMsgHandler_HandleImport_Call) RunAndReturn(run func(context.Context, string, *msgpb.ImportMsg) error) *MockMsgHandler_HandleImport_Call {
_c.Call.Return(run)
return _c
}
// HandleManualFlush provides a mock function with given fields: vchannel, flushMsg
func (_m *MockMsgHandler) HandleManualFlush(vchannel string, flushMsg message.ImmutableManualFlushMessageV2) error {
ret := _m.Called(vchannel, flushMsg)
if len(ret) == 0 {
panic("no return value specified for HandleManualFlush")
}
var r0 error
if rf, ok := ret.Get(0).(func(string, message.ImmutableManualFlushMessageV2) error); ok {
r0 = rf(vchannel, flushMsg)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockMsgHandler_HandleManualFlush_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleManualFlush'
type MockMsgHandler_HandleManualFlush_Call struct {
*mock.Call
}
// HandleManualFlush is a helper method to define mock.On call
// - vchannel string
// - flushMsg message.specializedImmutableMessage[*messagespb.ManualFlushMessageHeader,*messagespb.ManualFlushMessageBody]
func (_e *MockMsgHandler_Expecter) HandleManualFlush(vchannel interface{}, flushMsg interface{}) *MockMsgHandler_HandleManualFlush_Call {
return &MockMsgHandler_HandleManualFlush_Call{Call: _e.mock.On("HandleManualFlush", vchannel, flushMsg)}
}
func (_c *MockMsgHandler_HandleManualFlush_Call) Run(run func(vchannel string, flushMsg message.ImmutableManualFlushMessageV2)) *MockMsgHandler_HandleManualFlush_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(string), args[1].(message.ImmutableManualFlushMessageV2))
})
return _c
}
func (_c *MockMsgHandler_HandleManualFlush_Call) Return(_a0 error) *MockMsgHandler_HandleManualFlush_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockMsgHandler_HandleManualFlush_Call) RunAndReturn(run func(string, message.ImmutableManualFlushMessageV2) error) *MockMsgHandler_HandleManualFlush_Call {
_c.Call.Return(run)
return _c
}
// NewMockMsgHandler creates a new instance of MockMsgHandler. 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 NewMockMsgHandler(t interface {
mock.TestingT
Cleanup(func())
}) *MockMsgHandler {
mock := &MockMsgHandler{}
mock.Mock.Test(t)
t.Cleanup(func() { mock.AssertExpectations(t) })
return mock
}

View File

@ -1,180 +0,0 @@
// Code generated by mockery v2.46.0. DO NOT EDIT.
package mock_flusher
import (
context "context"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
mock "github.com/stretchr/testify/mock"
)
// MockFlushMsgHandler is an autogenerated mock type for the FlushMsgHandler type
type MockFlushMsgHandler struct {
mock.Mock
}
type MockFlushMsgHandler_Expecter struct {
mock *mock.Mock
}
func (_m *MockFlushMsgHandler) EXPECT() *MockFlushMsgHandler_Expecter {
return &MockFlushMsgHandler_Expecter{mock: &_m.Mock}
}
// HandleCreateSegment provides a mock function with given fields: ctx, vchannel, createSegmentMsg
func (_m *MockFlushMsgHandler) HandleCreateSegment(ctx context.Context, vchannel string, createSegmentMsg message.ImmutableCreateSegmentMessageV2) error {
ret := _m.Called(ctx, vchannel, createSegmentMsg)
if len(ret) == 0 {
panic("no return value specified for HandleCreateSegment")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, string, message.ImmutableCreateSegmentMessageV2) error); ok {
r0 = rf(ctx, vchannel, createSegmentMsg)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockFlushMsgHandler_HandleCreateSegment_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleCreateSegment'
type MockFlushMsgHandler_HandleCreateSegment_Call struct {
*mock.Call
}
// HandleCreateSegment is a helper method to define mock.On call
// - ctx context.Context
// - vchannel string
// - createSegmentMsg message.specializedImmutableMessage[*messagespb.CreateSegmentMessageHeader,*messagespb.CreateSegmentMessageBody]
func (_e *MockFlushMsgHandler_Expecter) HandleCreateSegment(ctx interface{}, vchannel interface{}, createSegmentMsg interface{}) *MockFlushMsgHandler_HandleCreateSegment_Call {
return &MockFlushMsgHandler_HandleCreateSegment_Call{Call: _e.mock.On("HandleCreateSegment", ctx, vchannel, createSegmentMsg)}
}
func (_c *MockFlushMsgHandler_HandleCreateSegment_Call) Run(run func(ctx context.Context, vchannel string, createSegmentMsg message.ImmutableCreateSegmentMessageV2)) *MockFlushMsgHandler_HandleCreateSegment_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(string), args[2].(message.ImmutableCreateSegmentMessageV2))
})
return _c
}
func (_c *MockFlushMsgHandler_HandleCreateSegment_Call) Return(_a0 error) *MockFlushMsgHandler_HandleCreateSegment_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockFlushMsgHandler_HandleCreateSegment_Call) RunAndReturn(run func(context.Context, string, message.ImmutableCreateSegmentMessageV2) error) *MockFlushMsgHandler_HandleCreateSegment_Call {
_c.Call.Return(run)
return _c
}
// HandleFlush provides a mock function with given fields: vchannel, flushMsg
func (_m *MockFlushMsgHandler) HandleFlush(vchannel string, flushMsg message.ImmutableFlushMessageV2) error {
ret := _m.Called(vchannel, flushMsg)
if len(ret) == 0 {
panic("no return value specified for HandleFlush")
}
var r0 error
if rf, ok := ret.Get(0).(func(string, message.ImmutableFlushMessageV2) error); ok {
r0 = rf(vchannel, flushMsg)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockFlushMsgHandler_HandleFlush_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleFlush'
type MockFlushMsgHandler_HandleFlush_Call struct {
*mock.Call
}
// HandleFlush is a helper method to define mock.On call
// - vchannel string
// - flushMsg message.specializedImmutableMessage[*messagespb.FlushMessageHeader,*messagespb.FlushMessageBody]
func (_e *MockFlushMsgHandler_Expecter) HandleFlush(vchannel interface{}, flushMsg interface{}) *MockFlushMsgHandler_HandleFlush_Call {
return &MockFlushMsgHandler_HandleFlush_Call{Call: _e.mock.On("HandleFlush", vchannel, flushMsg)}
}
func (_c *MockFlushMsgHandler_HandleFlush_Call) Run(run func(vchannel string, flushMsg message.ImmutableFlushMessageV2)) *MockFlushMsgHandler_HandleFlush_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(string), args[1].(message.ImmutableFlushMessageV2))
})
return _c
}
func (_c *MockFlushMsgHandler_HandleFlush_Call) Return(_a0 error) *MockFlushMsgHandler_HandleFlush_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockFlushMsgHandler_HandleFlush_Call) RunAndReturn(run func(string, message.ImmutableFlushMessageV2) error) *MockFlushMsgHandler_HandleFlush_Call {
_c.Call.Return(run)
return _c
}
// HandleManualFlush provides a mock function with given fields: vchannel, flushMsg
func (_m *MockFlushMsgHandler) HandleManualFlush(vchannel string, flushMsg message.ImmutableManualFlushMessageV2) error {
ret := _m.Called(vchannel, flushMsg)
if len(ret) == 0 {
panic("no return value specified for HandleManualFlush")
}
var r0 error
if rf, ok := ret.Get(0).(func(string, message.ImmutableManualFlushMessageV2) error); ok {
r0 = rf(vchannel, flushMsg)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockFlushMsgHandler_HandleManualFlush_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleManualFlush'
type MockFlushMsgHandler_HandleManualFlush_Call struct {
*mock.Call
}
// HandleManualFlush is a helper method to define mock.On call
// - vchannel string
// - flushMsg message.specializedImmutableMessage[*messagespb.ManualFlushMessageHeader,*messagespb.ManualFlushMessageBody]
func (_e *MockFlushMsgHandler_Expecter) HandleManualFlush(vchannel interface{}, flushMsg interface{}) *MockFlushMsgHandler_HandleManualFlush_Call {
return &MockFlushMsgHandler_HandleManualFlush_Call{Call: _e.mock.On("HandleManualFlush", vchannel, flushMsg)}
}
func (_c *MockFlushMsgHandler_HandleManualFlush_Call) Run(run func(vchannel string, flushMsg message.ImmutableManualFlushMessageV2)) *MockFlushMsgHandler_HandleManualFlush_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(string), args[1].(message.ImmutableManualFlushMessageV2))
})
return _c
}
func (_c *MockFlushMsgHandler_HandleManualFlush_Call) Return(_a0 error) *MockFlushMsgHandler_HandleManualFlush_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockFlushMsgHandler_HandleManualFlush_Call) RunAndReturn(run func(string, message.ImmutableManualFlushMessageV2) error) *MockFlushMsgHandler_HandleManualFlush_Call {
_c.Call.Return(run)
return _c
}
// NewMockFlushMsgHandler creates a new instance of MockFlushMsgHandler. 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 NewMockFlushMsgHandler(t interface {
mock.TestingT
Cleanup(func())
}) *MockFlushMsgHandler {
mock := &MockFlushMsgHandler{}
mock.Mock.Test(t)
t.Cleanup(func() { mock.AssertExpectations(t) })
return mock
}

View File

@ -41,12 +41,12 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"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/distributed/streaming"
"github.com/milvus-io/milvus/internal/http"
"github.com/milvus-io/milvus/internal/proxy/connection"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/ctokenizer"
"github.com/milvus-io/milvus/internal/util/hookutil"
"github.com/milvus-io/milvus/internal/util/importutilv2"
"github.com/milvus-io/milvus/internal/util/streamingutil"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
@ -56,6 +56,7 @@ import (
"github.com/milvus-io/milvus/pkg/proto/internalpb"
"github.com/milvus-io/milvus/pkg/proto/proxypb"
"github.com/milvus-io/milvus/pkg/proto/querypb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/util"
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
"github.com/milvus-io/milvus/pkg/util/crypto"
@ -6535,143 +6536,61 @@ func (node *Proxy) ImportV2(ctx context.Context, req *internalpb.ImportRequest)
method := "ImportV2"
tr := timerecord.NewTimeRecorder(method)
log.Info(rpcReceived(method))
nodeID := fmt.Sprint(paramtable.GetNodeID())
defer func() {
metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.TotalLabel, req.GetDbName(), req.GetCollectionName()).Inc()
if resp.GetStatus().GetCode() != 0 {
log.Warn("import failed", zap.String("err", resp.GetStatus().GetReason()))
metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.FailLabel, req.GetDbName(), req.GetCollectionName()).Inc()
} else {
metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.SuccessLabel, req.GetDbName(), req.GetCollectionName()).Inc()
}
}()
metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.TotalLabel, req.GetDbName(), req.GetCollectionName()).Inc()
collectionID, err := globalMetaCache.GetCollectionID(ctx, req.GetDbName(), req.GetCollectionName())
if err != nil {
resp.Status = merr.Status(err)
return resp, nil
it := &importTask{
ctx: ctx,
Condition: NewTaskCondition(ctx),
req: req,
node: node,
dataCoord: node.dataCoord,
resp: resp,
}
schema, err := globalMetaCache.GetCollectionSchema(ctx, req.GetDbName(), req.GetCollectionName())
if err != nil {
resp.Status = merr.Status(err)
return resp, nil
}
channels, err := node.chMgr.getVChannels(collectionID)
if err != nil {
if err := node.sched.dmQueue.Enqueue(it); err != nil {
log.Warn(
rpcFailedToEnqueue(method),
zap.Error(err))
metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.AbandonLabel, req.GetDbName(), req.GetCollectionName()).Inc()
resp.Status = merr.Status(err)
return resp, nil
}
isBackup := importutilv2.IsBackup(req.GetOptions())
isL0Import := importutilv2.IsL0Import(req.GetOptions())
hasPartitionKey := typeutil.HasPartitionKey(schema.CollectionSchema)
log.Info(
rpcEnqueued(method),
zap.Uint64("BeginTs", it.BeginTs()),
zap.Uint64("EndTs", it.EndTs()))
var partitionIDs []int64
if isBackup {
if req.GetPartitionName() == "" {
resp.Status = merr.Status(merr.WrapErrParameterInvalidMsg("partition not specified"))
return resp, nil
}
// Currently, Backup tool call import must with a partition name, each time restore a partition
partitionID, err := globalMetaCache.GetPartitionID(ctx, req.GetDbName(), req.GetCollectionName(), req.GetPartitionName())
if err != nil {
resp.Status = merr.Status(err)
return resp, nil
}
partitionIDs = []UniqueID{partitionID}
} else if isL0Import {
if req.GetPartitionName() == "" {
partitionIDs = []UniqueID{common.AllPartitionsID}
} else {
partitionID, err := globalMetaCache.GetPartitionID(ctx, req.GetDbName(), req.GetCollectionName(), req.PartitionName)
if err != nil {
resp.Status = merr.Status(err)
return resp, nil
}
partitionIDs = []UniqueID{partitionID}
}
// Currently, querynodes first load L0 segments and then load L1 segments.
// Therefore, to ensure the deletes from L0 import take effect,
// the collection needs to be in an unloaded state,
// and then all L0 and L1 segments should be loaded at once.
// We will remove this restriction after querynode supported to load L0 segments dynamically.
loaded, err := isCollectionLoaded(ctx, node.queryCoord, collectionID)
if err != nil {
resp.Status = merr.Status(err)
return resp, nil
}
if loaded {
resp.Status = merr.Status(merr.WrapErrImportFailed("for l0 import, collection cannot be loaded, please release it first"))
return resp, nil
}
} else {
if hasPartitionKey {
if req.GetPartitionName() != "" {
resp.Status = merr.Status(merr.WrapErrImportFailed("not allow to set partition name for collection with partition key"))
return resp, nil
}
partitions, err := globalMetaCache.GetPartitions(ctx, req.GetDbName(), req.GetCollectionName())
if err != nil {
resp.Status = merr.Status(err)
return resp, nil
}
_, partitionIDs, err = typeutil.RearrangePartitionsForPartitionKey(partitions)
if err != nil {
resp.Status = merr.Status(err)
return resp, nil
}
} else {
if req.GetPartitionName() == "" {
req.PartitionName = Params.CommonCfg.DefaultPartitionName.GetValue()
}
partitionID, err := globalMetaCache.GetPartitionID(ctx, req.GetDbName(), req.GetCollectionName(), req.PartitionName)
if err != nil {
resp.Status = merr.Status(err)
return resp, nil
}
partitionIDs = []UniqueID{partitionID}
}
}
req.Files = lo.Filter(req.GetFiles(), func(file *internalpb.ImportFile, _ int) bool {
return len(file.GetPaths()) > 0
})
if len(req.Files) == 0 {
resp.Status = merr.Status(merr.WrapErrParameterInvalidMsg("import request is empty"))
return resp, nil
}
if len(req.Files) > Params.DataCoordCfg.MaxFilesPerImportReq.GetAsInt() {
resp.Status = merr.Status(merr.WrapErrImportFailed(fmt.Sprintf("The max number of import files should not exceed %d, but got %d",
Params.DataCoordCfg.MaxFilesPerImportReq.GetAsInt(), len(req.Files))))
return resp, nil
}
if !isBackup && !isL0Import {
// check file type
for _, file := range req.GetFiles() {
_, err = importutilv2.GetFileType(file)
if err != nil {
resp.Status = merr.Status(err)
return resp, nil
}
}
}
importRequest := &internalpb.ImportRequestInternal{
CollectionID: collectionID,
CollectionName: req.GetCollectionName(),
PartitionIDs: partitionIDs,
ChannelNames: channels,
Schema: schema.CollectionSchema,
Files: req.GetFiles(),
Options: req.GetOptions(),
}
resp, err = node.dataCoord.ImportV2(ctx, importRequest)
if err != nil {
log.Warn("import failed", zap.Error(err))
if err := it.WaitToFinish(); err != nil {
log.Warn(
rpcFailedToWaitToFinish(method),
zap.Error(err),
zap.Uint64("BeginTs", it.BeginTs()),
zap.Uint64("EndTs", it.EndTs()))
metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.FailLabel, req.GetDbName(), req.GetCollectionName()).Inc()
resp.Status = merr.Status(err)
return resp, nil
}
// Import is asynchronous consumed from the wal, so we need to wait for the wal to release the resource key.
// The job can be seen by the user after the resource key is acked once at any vchannel.
jobID, err := strconv.ParseInt(resp.GetJobID(), 10, 64)
if err != nil {
return &internalpb.ImportResponse{
Status: merr.Status(merr.WrapErrServiceInternal("invalid job ID")),
}, nil
}
resourceKey := message.NewImportJobIDResourceKey(jobID)
if err := streaming.WAL().Broadcast().BlockUntilResourceKeyAckOnce(ctx, resourceKey); err != nil {
log.Warn("failed to wait for resource key ack", zap.Error(err))
return &internalpb.ImportResponse{
Status: merr.Status(merr.WrapErrServiceInternal("failed to wait for resource key ack")),
}, nil
}
metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.SuccessLabel, req.GetDbName(), req.GetCollectionName()).Inc()
metrics.ProxyReqLatency.WithLabelValues(nodeID, method).Observe(float64(tr.ElapseSpan().Milliseconds()))
return resp, err
return resp, nil
}
func (node *Proxy) GetImportProgress(ctx context.Context, req *internalpb.GetImportProgressRequest) (*internalpb.GetImportProgressResponse, error) {

View File

@ -20,6 +20,7 @@ import (
"context"
"encoding/base64"
"fmt"
"math/rand"
"net/http"
"net/http/httptest"
"testing"
@ -40,8 +41,10 @@ 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/allocator"
"github.com/milvus-io/milvus/internal/distributed/streaming"
mhttp "github.com/milvus-io/milvus/internal/http"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/mocks/distributed/mock_streaming"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/pkg/common"
@ -53,6 +56,7 @@ import (
"github.com/milvus-io/milvus/pkg/proto/proxypb"
"github.com/milvus-io/milvus/pkg/proto/querypb"
"github.com/milvus-io/milvus/pkg/proto/rootcoordpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"
@ -1610,6 +1614,13 @@ func TestProxy_ReplicateMessage(t *testing.T) {
}
func TestProxy_ImportV2(t *testing.T) {
wal := mock_streaming.NewMockWALAccesser(t)
b := mock_streaming.NewMockBroadcast(t)
wal.EXPECT().Broadcast().Return(b).Maybe()
b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe()
b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil).Maybe()
streaming.SetWALForTest(wal)
defer streaming.RecoverWALForTest()
ctx := context.Background()
mockErr := errors.New("mock error")
@ -1625,6 +1636,22 @@ func TestProxy_ImportV2(t *testing.T) {
assert.NotEqual(t, int32(0), rsp.GetStatus().GetCode())
node.UpdateStateCode(commonpb.StateCode_Healthy)
factory := dependency.NewDefaultFactory(true)
node, err = NewProxy(ctx, factory)
assert.NoError(t, err)
node.UpdateStateCode(commonpb.StateCode_Healthy)
node.tsoAllocator = &timestampAllocator{
tso: newMockTimestampAllocatorInterface(),
}
scheduler, err := newTaskScheduler(ctx, node.tsoAllocator, factory)
assert.NoError(t, err)
node.sched = scheduler
err = node.sched.Start()
assert.NoError(t, err)
chMgr := NewMockChannelsMgr(t)
chMgr.EXPECT().getChannels(mock.Anything).Return([]string{"p1"}, nil)
node.chMgr = chMgr
// no such collection
mc := NewMockCache(t)
mc.EXPECT().GetCollectionID(mock.Anything, mock.Anything, mock.Anything).Return(0, mockErr)
@ -1651,9 +1678,7 @@ func TestProxy_ImportV2(t *testing.T) {
}},
}, nil)
globalMetaCache = mc
chMgr := NewMockChannelsMgr(t)
chMgr.EXPECT().getVChannels(mock.Anything).Return(nil, mockErr)
node.chMgr = chMgr
chMgr.EXPECT().getVChannels(mock.Anything).Return(nil, mockErr).Once()
rsp, err = node.ImportV2(ctx, &internalpb.ImportRequest{CollectionName: "aaa"})
assert.NoError(t, err)
assert.NotEqual(t, int32(0), rsp.GetStatus().GetCode())
@ -1661,6 +1686,7 @@ func TestProxy_ImportV2(t *testing.T) {
// set partition name and with partition key
chMgr = NewMockChannelsMgr(t)
chMgr.EXPECT().getVChannels(mock.Anything).Return([]string{"ch0"}, nil)
chMgr.EXPECT().getChannels(mock.Anything).Return([]string{"p1"}, nil)
node.chMgr = chMgr
rsp, err = node.ImportV2(ctx, &internalpb.ImportRequest{CollectionName: "aaa", PartitionName: "bbb"})
assert.NoError(t, err)
@ -1717,9 +1743,17 @@ func TestProxy_ImportV2(t *testing.T) {
assert.NotEqual(t, int32(0), rsp.GetStatus().GetCode())
// normal case
dataCoord := mocks.NewMockDataCoordClient(t)
dataCoord.EXPECT().ImportV2(mock.Anything, mock.Anything).Return(nil, nil)
node.dataCoord = dataCoord
rc := mocks.NewMockRootCoordClient(t)
rc.EXPECT().AllocID(mock.Anything, mock.Anything).Return(&rootcoordpb.AllocIDResponse{
ID: rand.Int63(),
Count: 1,
}, nil).Once()
idAllocator, err := allocator.NewIDAllocator(ctx, rc, 0)
assert.NoError(t, err)
node.rowIDAllocator = idAllocator
err = idAllocator.Start()
assert.NoError(t, err)
rsp, err = node.ImportV2(ctx, &internalpb.ImportRequest{
CollectionName: "aaa",
Files: []*internalpb.ImportFile{{

View File

@ -43,6 +43,7 @@ import (
"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/allocator"
"github.com/milvus-io/milvus/internal/coordinator/coordclient"
grpcdatacoordclient "github.com/milvus-io/milvus/internal/distributed/datacoord"
grpcdatacoordclient2 "github.com/milvus-io/milvus/internal/distributed/datacoord/client"
@ -53,8 +54,10 @@ import (
grpcquerynode "github.com/milvus-io/milvus/internal/distributed/querynode"
grpcrootcoord "github.com/milvus-io/milvus/internal/distributed/rootcoord"
rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client"
"github.com/milvus-io/milvus/internal/distributed/streaming"
"github.com/milvus-io/milvus/internal/json"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/mocks/distributed/mock_streaming"
"github.com/milvus-io/milvus/internal/util/componentutil"
"github.com/milvus-io/milvus/internal/util/dependency"
"github.com/milvus-io/milvus/internal/util/sessionutil"
@ -65,6 +68,7 @@ import (
"github.com/milvus-io/milvus/pkg/proto/proxypb"
"github.com/milvus-io/milvus/pkg/proto/querypb"
"github.com/milvus-io/milvus/pkg/proto/rootcoordpb"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/tracer"
"github.com/milvus-io/milvus/pkg/util"
"github.com/milvus-io/milvus/pkg/util/crypto"
@ -2020,22 +2024,6 @@ func TestProxy(t *testing.T) {
rateCol.Register(internalpb.RateType_DMLInsert.String())
})
wg.Add(1)
t.Run("test import", func(t *testing.T) {
defer wg.Done()
req := &milvuspb.ImportRequest{
DbName: dbName,
CollectionName: collectionName,
Files: []string{"f1.json"},
}
proxy.UpdateStateCode(commonpb.StateCode_Healthy)
resp, err := proxy.Import(context.TODO(), req)
assert.EqualValues(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
assert.NoError(t, err)
// Wait a bit for complete import to start.
time.Sleep(2 * time.Second)
})
wg.Add(1)
t.Run("release collection", func(t *testing.T) {
defer wg.Done()
@ -4669,6 +4657,13 @@ func TestProxy_Import(t *testing.T) {
cache := globalMetaCache
defer func() { globalMetaCache = cache }()
wal := mock_streaming.NewMockWALAccesser(t)
b := mock_streaming.NewMockBroadcast(t)
wal.EXPECT().Broadcast().Return(b).Maybe()
b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe()
streaming.SetWALForTest(wal)
defer streaming.RecoverWALForTest()
t.Run("Import failed", func(t *testing.T) {
proxy := &Proxy{}
proxy.UpdateStateCode(commonpb.StateCode_Abnormal)
@ -4680,6 +4675,7 @@ func TestProxy_Import(t *testing.T) {
})
t.Run("Import", func(t *testing.T) {
ctx := context.Background()
proxy := &Proxy{}
proxy.UpdateStateCode(commonpb.StateCode_Healthy)
@ -4692,15 +4688,37 @@ func TestProxy_Import(t *testing.T) {
globalMetaCache = mc
chMgr := NewMockChannelsMgr(t)
chMgr.EXPECT().getVChannels(mock.Anything).Return(nil, nil)
chMgr.EXPECT().getVChannels(mock.Anything).Return([]string{"foo"}, nil)
chMgr.EXPECT().getChannels(mock.Anything).Return([]string{"foo_v1"}, nil)
proxy.chMgr = chMgr
dataCoord := mocks.NewMockDataCoordClient(t)
dataCoord.EXPECT().ImportV2(mock.Anything, mock.Anything).Return(&internalpb.ImportResponse{
Status: merr.Success(),
JobID: "100",
}, nil)
proxy.dataCoord = dataCoord
factory := dependency.NewDefaultFactory(true)
rc := mocks.NewMockRootCoordClient(t)
rc.EXPECT().AllocID(mock.Anything, mock.Anything).Return(&rootcoordpb.AllocIDResponse{
ID: rand.Int63(),
Count: 1,
}, nil).Once()
idAllocator, err := allocator.NewIDAllocator(ctx, rc, 0)
assert.NoError(t, err)
err = idAllocator.Start()
assert.NoError(t, err)
proxy.rowIDAllocator = idAllocator
proxy.tsoAllocator = &timestampAllocator{
tso: newMockTimestampAllocatorInterface(),
}
scheduler, err := newTaskScheduler(ctx, proxy.tsoAllocator, factory)
assert.NoError(t, err)
proxy.sched = scheduler
err = proxy.sched.Start()
assert.NoError(t, err)
wal := mock_streaming.NewMockWALAccesser(t)
b := mock_streaming.NewMockBroadcast(t)
wal.EXPECT().Broadcast().Return(b)
b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil)
b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe()
streaming.SetWALForTest(wal)
defer streaming.RecoverWALForTest()
req := &milvuspb.ImportRequest{
CollectionName: "dummy",

View File

@ -0,0 +1,273 @@
/*
* 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 proxy
import (
"context"
"fmt"
"strconv"
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/internal/distributed/streaming"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/importutilv2"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/proto/internalpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
type importTask struct {
baseTask
Condition
req *internalpb.ImportRequest
ctx context.Context
node *Proxy
dataCoord types.DataCoordClient
msgID UniqueID
taskTS Timestamp
vchannels []string
pchannels []string
partitionIDs []int64
collectionID UniqueID
schema *schemaInfo
resp *internalpb.ImportResponse
}
func (it *importTask) TraceCtx() context.Context {
return it.ctx
}
func (it *importTask) ID() UniqueID {
return it.msgID
}
func (it *importTask) SetID(uid UniqueID) {
it.msgID = uid
}
func (it *importTask) Name() string {
return "ImportTask"
}
func (it *importTask) Type() commonpb.MsgType {
return commonpb.MsgType_Import
}
func (it *importTask) BeginTs() Timestamp {
return it.taskTS
}
func (it *importTask) EndTs() Timestamp {
return it.taskTS
}
func (it *importTask) SetTs(ts Timestamp) {
it.taskTS = ts
}
func (it *importTask) OnEnqueue() error {
return nil
}
func (it *importTask) PreExecute(ctx context.Context) error {
req := it.req
node := it.node
collectionID, err := globalMetaCache.GetCollectionID(ctx, req.GetDbName(), req.GetCollectionName())
if err != nil {
return err
}
it.collectionID = collectionID
schema, err := globalMetaCache.GetCollectionSchema(ctx, req.GetDbName(), req.GetCollectionName())
if err != nil {
return err
}
it.schema = schema
channels, err := node.chMgr.getVChannels(collectionID)
if err != nil {
return err
}
it.vchannels = channels
isBackup := importutilv2.IsBackup(req.GetOptions())
isL0Import := importutilv2.IsL0Import(req.GetOptions())
hasPartitionKey := typeutil.HasPartitionKey(schema.CollectionSchema)
var partitionIDs []int64
if isBackup {
if req.GetPartitionName() == "" {
return merr.WrapErrParameterInvalidMsg("partition not specified")
}
// Currently, Backup tool call import must with a partition name, each time restore a partition
partitionID, err := globalMetaCache.GetPartitionID(ctx, req.GetDbName(), req.GetCollectionName(), req.GetPartitionName())
if err != nil {
return err
}
partitionIDs = []UniqueID{partitionID}
} else if isL0Import {
if req.GetPartitionName() == "" {
partitionIDs = []UniqueID{common.AllPartitionsID}
} else {
partitionID, err := globalMetaCache.GetPartitionID(ctx, req.GetDbName(), req.GetCollectionName(), req.PartitionName)
if err != nil {
return err
}
partitionIDs = []UniqueID{partitionID}
}
// Currently, querynodes first load L0 segments and then load L1 segments.
// Therefore, to ensure the deletes from L0 import take effect,
// the collection needs to be in an unloaded state,
// and then all L0 and L1 segments should be loaded at once.
// We will remove this restriction after querynode supported to load L0 segments dynamically.
loaded, err := isCollectionLoaded(ctx, node.queryCoord, collectionID)
if err != nil {
return err
}
if loaded {
return merr.WrapErrImportFailed("for l0 import, collection cannot be loaded, please release it first")
}
} else {
if hasPartitionKey {
if req.GetPartitionName() != "" {
return merr.WrapErrImportFailed("not allow to set partition name for collection with partition key")
}
partitions, err := globalMetaCache.GetPartitions(ctx, req.GetDbName(), req.GetCollectionName())
if err != nil {
return err
}
_, partitionIDs, err = typeutil.RearrangePartitionsForPartitionKey(partitions)
if err != nil {
return err
}
} else {
if req.GetPartitionName() == "" {
req.PartitionName = Params.CommonCfg.DefaultPartitionName.GetValue()
}
partitionID, err := globalMetaCache.GetPartitionID(ctx, req.GetDbName(), req.GetCollectionName(), req.PartitionName)
if err != nil {
return err
}
partitionIDs = []UniqueID{partitionID}
}
}
req.Files = lo.Filter(req.GetFiles(), func(file *internalpb.ImportFile, _ int) bool {
return len(file.GetPaths()) > 0
})
if len(req.Files) == 0 {
return merr.WrapErrParameterInvalidMsg("import request is empty")
}
if len(req.Files) > Params.DataCoordCfg.MaxFilesPerImportReq.GetAsInt() {
return merr.WrapErrImportFailed(fmt.Sprintf("The max number of import files should not exceed %d, but got %d",
Params.DataCoordCfg.MaxFilesPerImportReq.GetAsInt(), len(req.Files)))
}
if !isBackup && !isL0Import {
// check file type
for _, file := range req.GetFiles() {
_, err = importutilv2.GetFileType(file)
if err != nil {
return err
}
}
}
it.partitionIDs = partitionIDs
return nil
}
func (it *importTask) setChannels() error {
collID, err := globalMetaCache.GetCollectionID(it.ctx, it.req.GetDbName(), it.req.CollectionName)
if err != nil {
return err
}
channels, err := it.node.chMgr.getChannels(collID)
if err != nil {
return err
}
it.pchannels = channels
return nil
}
func (it *importTask) getChannels() []pChan {
return it.pchannels
}
func (it *importTask) Execute(ctx context.Context) error {
jobID, err := it.node.rowIDAllocator.AllocOne()
if err != nil {
log.Ctx(ctx).Warn("alloc job id failed", zap.Error(err))
return err
}
resourceKey := message.NewImportJobIDResourceKey(jobID)
msg, err := message.NewImportMessageBuilderV1().
WithHeader(&message.ImportMessageHeader{}).WithBody(
&msgpb.ImportMsg{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Import,
Timestamp: it.BeginTs(),
},
DbName: it.req.GetDbName(),
CollectionName: it.req.GetCollectionName(),
CollectionID: it.collectionID,
PartitionIDs: it.partitionIDs,
Options: funcutil.KeyValuePair2Map(it.req.GetOptions()),
Files: GetImportFiles(it.req.GetFiles()),
Schema: it.schema.CollectionSchema,
JobID: jobID,
}).
WithBroadcast(it.vchannels, resourceKey).
BuildBroadcast()
if err != nil {
log.Ctx(ctx).Warn("create import message failed", zap.Error(err))
return err
}
resp, err := streaming.WAL().Broadcast().Append(ctx, msg)
if err != nil {
log.Ctx(ctx).Warn("broadcast import msg failed", zap.Error(err))
return err
}
log.Ctx(ctx).Info(
"broadcast import msg success",
zap.Int64("jobID", jobID),
zap.Uint64("broadcastID", resp.BroadcastID),
zap.Any("appendResults", resp.AppendResults),
)
it.resp.JobID = strconv.FormatInt(jobID, 10)
return nil
}
func GetImportFiles(internals []*internalpb.ImportFile) []*msgpb.ImportFile {
return lo.Map(internals, func(internal *internalpb.ImportFile, _ int) *msgpb.ImportFile {
return &msgpb.ImportFile{
Id: internal.GetId(),
Paths: internal.GetPaths(),
}
})
}
func (it *importTask) PostExecute(ctx context.Context) error {
return nil
}

View File

@ -63,12 +63,10 @@ type broadcastTaskManager struct {
// AddTask adds a new broadcast task into the manager.
func (bm *broadcastTaskManager) AddTask(ctx context.Context, msg message.BroadcastMutableMessage) (*pendingBroadcastTask, error) {
id, err := resource.Resource().IDAllocator().Allocate(ctx)
if err != nil {
return nil, errors.Wrapf(err, "allocate new id failed")
var err error
if msg, err = bm.assignID(ctx, msg); err != nil {
return nil, err
}
msg = msg.WithBroadcastID(id)
task, err := bm.addBroadcastTask(msg)
if err != nil {
return nil, err
@ -76,6 +74,30 @@ func (bm *broadcastTaskManager) AddTask(ctx context.Context, msg message.Broadca
return newPendingBroadcastTask(task), nil
}
func (bm *broadcastTaskManager) assignID(ctx context.Context, msg message.BroadcastMutableMessage) (message.BroadcastMutableMessage, error) {
// TODO: current implementation the header cannot be seen at flusher itself.
// only import message use it, so temporarily set the broadcast id here.
// need to refactor the message to make the broadcast header visible to flusher.
if msg.MessageType() == message.MessageTypeImport {
importMsg, err := message.AsMutableImportMessageV1(msg)
if err != nil {
return nil, err
}
body, err := importMsg.Body()
if err != nil {
return nil, err
}
return msg.WithBroadcastID(uint64(body.JobID)), nil
}
id, err := resource.Resource().IDAllocator().Allocate(ctx)
if err != nil {
return nil, errors.Wrapf(err, "allocate new id failed")
}
msg = msg.WithBroadcastID(id)
return msg, nil
}
// Ack acknowledges the message at the specified vchannel.
func (bm *broadcastTaskManager) Ack(ctx context.Context, broadcastID uint64, vchannel string) error {
task, ok := bm.getBroadcastTaskByID(broadcastID)

View File

@ -20,11 +20,21 @@ import (
"context"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus/internal/distributed/streaming"
"github.com/milvus-io/milvus/internal/flushcommon/util"
"github.com/milvus-io/milvus/internal/flushcommon/writebuffer"
"github.com/milvus-io/milvus/internal/streamingnode/server/resource"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/proto/internalpb"
"github.com/milvus-io/milvus/pkg/streaming/util/message"
"github.com/milvus-io/milvus/pkg/streaming/util/types"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/retry"
)
func newMsgHandler(wbMgr writebuffer.BufferManager) *msgHandlerImpl {
@ -75,3 +85,42 @@ func (impl *msgHandlerImpl) HandleManualFlush(vchannel string, flushMsg message.
}
return nil
}
func (impl *msgHandlerImpl) HandleImport(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg) error {
return retry.Do(ctx, func() (err error) {
defer func() {
if err == nil {
err = streaming.WAL().Broadcast().Ack(ctx, types.BroadcastAckRequest{
BroadcastID: uint64(importMsg.GetJobID()),
VChannel: vchannel,
})
}
}()
client, err := resource.Resource().DataCoordClient().GetWithContext(ctx)
if err != nil {
return err
}
importResp, err := client.ImportV2(ctx, &internalpb.ImportRequestInternal{
CollectionID: importMsg.GetCollectionID(),
CollectionName: importMsg.GetCollectionName(),
PartitionIDs: importMsg.GetPartitionIDs(),
ChannelNames: []string{vchannel},
Schema: importMsg.GetSchema(),
Files: lo.Map(importMsg.GetFiles(), util.ConvertInternalImportFile),
Options: funcutil.Map2KeyValuePair(importMsg.GetOptions()),
DataTimestamp: importMsg.GetBase().GetTimestamp(),
JobID: importMsg.GetJobID(),
})
err = merr.CheckRPCCall(importResp, err)
if errors.Is(err, merr.ErrCollectionNotFound) {
log.Ctx(ctx).Warn("import message failed because of collection not found, skip it", zap.String("job_id", importResp.GetJobID()), zap.Error(err))
return nil
}
if err != nil {
log.Ctx(ctx).Warn("import message failed", zap.String("job_id", importResp.GetJobID()), zap.Error(err))
return err
}
log.Ctx(ctx).Info("import message handled", zap.String("job_id", importResp.GetJobID()))
return nil
}, retry.AttemptAlways())
}

View File

@ -15,7 +15,7 @@ require (
github.com/jolestar/go-commons-pool/v2 v2.1.2
github.com/json-iterator/go v1.1.12
github.com/klauspost/compress v1.17.7
github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250208062437-5af22aa4b559
github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250214033407-ad08272e542b
github.com/nats-io/nats-server/v2 v2.10.12
github.com/nats-io/nats.go v1.34.1
github.com/panjf2000/ants/v2 v2.7.2

View File

@ -492,8 +492,8 @@ github.com/milvus-io/cgosymbolizer v0.0.0-20240722103217-b7dee0e50119 h1:9VXijWu
github.com/milvus-io/cgosymbolizer v0.0.0-20240722103217-b7dee0e50119/go.mod h1:DvXTE/K/RtHehxU8/GtDs4vFtfw64jJ3PaCnFri8CRg=
github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b h1:TfeY0NxYxZzUfIfYe5qYDBzt4ZYRqzUjTR6CvUzjat8=
github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b/go.mod h1:iwW+9cWfIzzDseEBCCeDSN5SD16Tidvy8cwQ7ZY8Qj4=
github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250208062437-5af22aa4b559 h1:c8n10eBkYU/HYaDUNAaKog4aIA3ZHO+GL7bHN2Ug/MA=
github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250208062437-5af22aa4b559/go.mod h1:/6UT4zZl6awVeXLeE7UGDWZvXj3IWkRsh3mqsn0DiAs=
github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250214033407-ad08272e542b h1:s3gdV+iYJMvo9mydBJVAEA2Uaz29eIuUnQK867U3G8I=
github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250214033407-ad08272e542b/go.mod h1:/6UT4zZl6awVeXLeE7UGDWZvXj3IWkRsh3mqsn0DiAs=
github.com/milvus-io/pulsar-client-go v0.12.1 h1:O2JZp1tsYiO7C0MQ4hrUY/aJXnn2Gry6hpm7UodghmE=
github.com/milvus-io/pulsar-client-go v0.12.1/go.mod h1:dkutuH4oS2pXiGm+Ti7fQZ4MRjrMPZ8IJeEGAWMeckk=
github.com/minio/highwayhash v1.0.2 h1:Aak5U0nElisjDCfPSG79Tgzkn2gl66NxOMspRrKnA/g=

View File

@ -78,11 +78,12 @@ const (
Executing = "executing"
Done = "done"
ImportStagePending = "pending"
ImportStagePreImport = "preimport"
ImportStageImport = "import"
ImportStageStats = "stats"
ImportStageBuildIndex = "build_index"
ImportStagePending = "pending"
ImportStagePreImport = "preimport"
ImportStageImport = "import"
ImportStageStats = "stats"
ImportStageBuildIndex = "build_index"
ImportStageWaitL0Import = "wait_l0_import"
compactionTypeLabelName = "compaction_type"
isVectorFieldLabelName = "is_vector_field"

View File

@ -0,0 +1,78 @@
/*
* 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 msgstream
import (
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
)
type ImportMsg struct {
BaseMsg
*msgpb.ImportMsg
}
var _ TsMsg = (*ImportMsg)(nil)
func (i *ImportMsg) ID() UniqueID {
return i.Base.MsgID
}
func (i *ImportMsg) SetID(id UniqueID) {
i.Base.MsgID = id
}
func (i *ImportMsg) Type() MsgType {
return i.Base.MsgType
}
func (i *ImportMsg) SourceID() int64 {
return i.Base.SourceID
}
func (i *ImportMsg) Marshal(input TsMsg) (MarshalType, error) {
importMsg := input.(*ImportMsg)
mb, err := proto.Marshal(importMsg.ImportMsg)
if err != nil {
return nil, err
}
return mb, nil
}
func (i *ImportMsg) Unmarshal(input MarshalType) (TsMsg, error) {
importMsg := &msgpb.ImportMsg{}
in, err := convertToByteArray(input)
if err != nil {
return nil, err
}
err = proto.Unmarshal(in, importMsg)
if err != nil {
return nil, err
}
rr := &ImportMsg{ImportMsg: importMsg}
rr.BeginTimestamp = importMsg.GetBase().GetTimestamp()
rr.EndTimestamp = importMsg.GetBase().GetTimestamp()
return rr, nil
}
func (i *ImportMsg) Size() int {
return proto.Size(i.ImportMsg)
}

View File

@ -0,0 +1,64 @@
/*
* 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 msgstream
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
)
func TestImportMsg(t *testing.T) {
var msg TsMsg = &ImportMsg{
ImportMsg: &msgpb.ImportMsg{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Import,
MsgID: 100,
Timestamp: 1000,
SourceID: 10000,
TargetID: 100000,
ReplicateInfo: nil,
},
DbName: "unit_db",
},
}
assert.EqualValues(t, 100, msg.ID())
msg.SetID(200)
assert.EqualValues(t, 200, msg.ID())
assert.Equal(t, commonpb.MsgType_Import, msg.Type())
assert.EqualValues(t, 10000, msg.SourceID())
msgBytes, err := msg.Marshal(msg)
assert.NoError(t, err)
var newMsg TsMsg = &ImportMsg{}
_, err = newMsg.Unmarshal("1")
assert.Error(t, err)
newMsg, err = newMsg.Unmarshal(msgBytes)
assert.NoError(t, err)
assert.EqualValues(t, 200, newMsg.ID())
assert.EqualValues(t, 1000, newMsg.BeginTs())
assert.EqualValues(t, 1000, newMsg.EndTs())
assert.True(t, msg.Size() > 0)
}

View File

@ -50,7 +50,7 @@ func TestCreateIndex(t *testing.T) {
msgBytes, err := msg.Marshal(msg)
assert.NoError(t, err)
var newMsg TsMsg = &ReleaseCollectionMsg{}
var newMsg TsMsg = &CreateIndexMsg{}
_, err = newMsg.Unmarshal("1")
assert.Error(t, err)

View File

@ -0,0 +1,64 @@
/*
* 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 msgstream
import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
)
func TestReplicateMsg(t *testing.T) {
var msg TsMsg = &ReplicateMsg{
ReplicateMsg: &msgpb.ReplicateMsg{
Base: &commonpb.MsgBase{
MsgType: commonpb.MsgType_Replicate,
MsgID: 100,
Timestamp: 1000,
SourceID: 10000,
TargetID: 100000,
ReplicateInfo: nil,
},
Database: "unit_db",
},
}
assert.EqualValues(t, 100, msg.ID())
msg.SetID(200)
assert.EqualValues(t, 200, msg.ID())
assert.Equal(t, commonpb.MsgType_Replicate, msg.Type())
assert.EqualValues(t, 10000, msg.SourceID())
msgBytes, err := msg.Marshal(msg)
assert.NoError(t, err)
var newMsg TsMsg = &ReplicateMsg{}
_, err = newMsg.Unmarshal("1")
assert.Error(t, err)
newMsg, err = newMsg.Unmarshal(msgBytes)
assert.NoError(t, err)
assert.EqualValues(t, 200, newMsg.ID())
assert.EqualValues(t, 1000, newMsg.BeginTs())
assert.EqualValues(t, 1000, newMsg.EndTs())
assert.True(t, msg.Size() > 0)
}

View File

@ -85,6 +85,7 @@ func (pudf *ProtoUDFactory) NewUnmarshalDispatcher() *ProtoUnmarshalDispatcher {
operateUserRoleMsg := OperateUserRoleMsg{}
operatePrivilegeMsg := OperatePrivilegeMsg{}
replicateMsg := ReplicateMsg{}
importMsg := ImportMsg{}
p := &ProtoUnmarshalDispatcher{}
p.TempMap = make(map[commonpb.MsgType]UnmarshalFunc)
@ -115,6 +116,7 @@ func (pudf *ProtoUDFactory) NewUnmarshalDispatcher() *ProtoUnmarshalDispatcher {
p.TempMap[commonpb.MsgType_OperateUserRole] = operateUserRoleMsg.Unmarshal
p.TempMap[commonpb.MsgType_OperatePrivilege] = operatePrivilegeMsg.Unmarshal
p.TempMap[commonpb.MsgType_Replicate] = replicateMsg.Unmarshal
p.TempMap[commonpb.MsgType_Import] = importMsg.Unmarshal
return p
}

View File

@ -617,6 +617,8 @@ message CompactionPlan {
IDRange pre_allocated_segmentIDs = 18;
int64 slot_usage = 19;
int64 max_size = 20;
// bf path for importing
// collection is importing
}
message CompactionSegment {
@ -885,6 +887,8 @@ message ImportJob {
repeated internal.ImportFile files = 14;
repeated common.KeyValuePair options = 15;
string start_time = 16;
repeated string ready_vchannels = 17;
uint64 data_ts = 18;
}
enum ImportTaskStateV2 {
@ -895,6 +899,11 @@ enum ImportTaskStateV2 {
Completed = 4;
}
enum ImportTaskSourceV2 {
Request = 0;
L0Compaction = 1;
}
message PreImportTask {
int64 jobID = 1;
int64 taskID = 2;
@ -919,6 +928,7 @@ message ImportTaskV2 {
repeated ImportFileStats file_stats = 9;
repeated int64 stats_segmentIDs = 10;
string created_time = 11;
ImportTaskSourceV2 source = 12;
}
enum GcCommand {

File diff suppressed because it is too large Load Diff

View File

@ -323,7 +323,7 @@ message ImportFile {
}
message ImportRequestInternal {
int64 dbID = 1;
int64 dbID = 1 [deprecated=true];
int64 collectionID = 2;
string collection_name = 3;
repeated int64 partitionIDs = 4;
@ -331,6 +331,8 @@ message ImportRequestInternal {
schema.CollectionSchema schema = 6;
repeated ImportFile files = 7;
repeated common.KeyValuePair options = 8;
uint64 data_timestamp = 9;
int64 jobID = 10;
}
message ImportRequest {

View File

@ -2793,6 +2793,7 @@ type ImportRequestInternal struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// Deprecated: Marked as deprecated in internal.proto.
DbID int64 `protobuf:"varint,1,opt,name=dbID,proto3" json:"dbID,omitempty"`
CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"`
CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
@ -2801,6 +2802,8 @@ type ImportRequestInternal struct {
Schema *schemapb.CollectionSchema `protobuf:"bytes,6,opt,name=schema,proto3" json:"schema,omitempty"`
Files []*ImportFile `protobuf:"bytes,7,rep,name=files,proto3" json:"files,omitempty"`
Options []*commonpb.KeyValuePair `protobuf:"bytes,8,rep,name=options,proto3" json:"options,omitempty"`
DataTimestamp uint64 `protobuf:"varint,9,opt,name=data_timestamp,json=dataTimestamp,proto3" json:"data_timestamp,omitempty"`
JobID int64 `protobuf:"varint,10,opt,name=jobID,proto3" json:"jobID,omitempty"`
}
func (x *ImportRequestInternal) Reset() {
@ -2835,6 +2838,7 @@ func (*ImportRequestInternal) Descriptor() ([]byte, []int) {
return file_internal_proto_rawDescGZIP(), []int{31}
}
// Deprecated: Marked as deprecated in internal.proto.
func (x *ImportRequestInternal) GetDbID() int64 {
if x != nil {
return x.DbID
@ -2891,6 +2895,20 @@ func (x *ImportRequestInternal) GetOptions() []*commonpb.KeyValuePair {
return nil
}
func (x *ImportRequestInternal) GetDataTimestamp() uint64 {
if x != nil {
return x.DataTimestamp
}
return 0
}
func (x *ImportRequestInternal) GetJobID() int64 {
if x != nil {
return x.JobID
}
return 0
}
type ImportRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -3976,157 +3994,161 @@ var file_internal_proto_rawDesc = []byte{
0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x01, 0x72, 0x22, 0x32, 0x0a, 0x0a, 0x49, 0x6d, 0x70, 0x6f,
0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,
0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18,
0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x22, 0xf6, 0x02, 0x0a,
0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x22, 0xb7, 0x03, 0x0a,
0x15, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x6e,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x01,
0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f,
0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03,
0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x27,
0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d,
0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74,
0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69,
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70,
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x63,
0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03,
0x28, 0x09, 0x52, 0x0c, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x73,
0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x06, 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,
0x37, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21,
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c,
0x65, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69,
0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x16, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x01,
0x20, 0x01, 0x28, 0x03, 0x42, 0x02, 0x18, 0x01, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x22,
0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02,
0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x49, 0x44, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c,
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x70,
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28,
0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12,
0x23, 0x0a, 0x0d, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73,
0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e,
0x61, 0x6d, 0x65, 0x73, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x06,
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, 0x37, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03,
0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72,
0x74, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x07,
0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e,
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d,
0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72,
0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x64, 0x61, 0x74,
0x61, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x09, 0x20, 0x01, 0x28,
0x04, 0x52, 0x0d, 0x64, 0x61, 0x74, 0x61, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70,
0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52,
0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x22, 0xee, 0x01, 0x0a, 0x0d, 0x49, 0x6d, 0x70, 0x6f, 0x72,
0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e,
0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d,
0x65, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f,
0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c,
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61,
0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01,
0x28, 0x09, 0x52, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d,
0x65, 0x12, 0x37, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b,
0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46,
0x69, 0x6c, 0x65, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x6f, 0x70,
0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69,
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f,
0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x07,
0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x5b, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72,
0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61,
0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e,
0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x07, 0x6f, 0x70,
0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xee, 0x01, 0x0a, 0x0d, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61,
0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65,
0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e,
0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72,
0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28,
0x09, 0x52, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65,
0x12, 0x37, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32,
0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69,
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69,
0x6c, 0x65, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x6f, 0x70, 0x74,
0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c,
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e,
0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x07, 0x6f,
0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x5b, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74,
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74,
0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53,
0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x14, 0x0a,
0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f,
0x62, 0x49, 0x44, 0x22, 0x49, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74,
0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49,
0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x22, 0x81,
0x02, 0x0a, 0x12, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x50, 0x72, 0x6f,
0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6e, 0x61,
0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x4e, 0x61,
0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18,
0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x12,
0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52,
0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72,
0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72,
0x65, 0x73, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f,
0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70,
0x6c, 0x65, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74,
0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x23,
0x0a, 0x0d, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18,
0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x52,
0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77,
0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f,
0x77, 0x73, 0x22, 0xc6, 0x03, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74,
0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73,
0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02,
0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70,
0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61,
0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01,
0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72,
0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x70, 0x72,
0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52,
0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12,
0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65,
0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65,
0x54, 0x69, 0x6d, 0x65, 0x12, 0x52, 0x0a, 0x0f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x70, 0x72, 0x6f,
0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, 0x2e,
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74,
0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b,
0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x0e, 0x74, 0x61, 0x73, 0x6b, 0x50, 0x72,
0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6d, 0x70, 0x6f,
0x72, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52,
0x0c, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a,
0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28,
0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, 0x0a,
0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09,
0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x54, 0x0a, 0x1a, 0x4c,
0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49,
0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x22, 0x0a,
0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20,
0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49,
0x44, 0x22, 0x56, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61,
0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65,
0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e,
0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x86, 0x02, 0x0a, 0x13, 0x4c, 0x69,
0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x14,
0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a,
0x6f, 0x62, 0x49, 0x44, 0x22, 0x49, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72,
0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,
0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62,
0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x22,
0x81, 0x02, 0x0a, 0x12, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x50, 0x72,
0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6e,
0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x4e,
0x61, 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65,
0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65,
0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09,
0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x67,
0x72, 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x67,
0x72, 0x65, 0x73, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65,
0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d,
0x70, 0x6c, 0x65, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61,
0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12,
0x23, 0x0a, 0x0d, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73,
0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64,
0x52, 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f,
0x77, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52,
0x6f, 0x77, 0x73, 0x22, 0xc6, 0x03, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72,
0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28,
0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06,
0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x73,
0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x73, 0x12, 0x3d,
0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x25,
0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18,
0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d,
0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74,
0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20,
0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x70,
0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x70,
0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09,
0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65,
0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d,
0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74,
0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x52, 0x0a, 0x0f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x70, 0x72,
0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29,
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62,
0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x18, 0x0a,
0x07, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07,
0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x67, 0x72,
0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x70, 0x72, 0x6f,
0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28,
0x09, 0x52, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d,
0x65, 0x73, 0x2a, 0x45, 0x0a, 0x09, 0x52, 0x61, 0x74, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12,
0x0b, 0x0a, 0x07, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08,
0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x43, 0x6f,
0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x50, 0x61,
0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x2a, 0xb9, 0x01, 0x0a, 0x08, 0x52, 0x61,
0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f, 0x6c,
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x44, 0x44, 0x4c,
0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, 0x44,
0x44, 0x4c, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x44, 0x4c,
0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x03, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f,
0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d,
0x4c, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c,
0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b, 0x44, 0x4d, 0x4c, 0x42,
0x75, 0x6c, 0x6b, 0x4c, 0x6f, 0x61, 0x64, 0x10, 0x07, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x51, 0x4c,
0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x10, 0x08, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x51, 0x4c, 0x51,
0x75, 0x65, 0x72, 0x79, 0x10, 0x09, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, 0x55, 0x70, 0x73,
0x65, 0x72, 0x74, 0x10, 0x0a, 0x2a, 0x81, 0x01, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74,
0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65,
0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12,
0x10, 0x0a, 0x0c, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10,
0x02, 0x12, 0x0d, 0x0a, 0x09, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x03,
0x12, 0x0a, 0x0a, 0x06, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09,
0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x49,
0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x12, 0x09,
0x0a, 0x05, 0x53, 0x74, 0x61, 0x74, 0x73, 0x10, 0x07, 0x42, 0x32, 0x5a, 0x30, 0x67, 0x69, 0x74,
0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69,
0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f,
0x74, 0x6f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x62, 0x06, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x33,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73,
0x6b, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x0e, 0x74, 0x61, 0x73, 0x6b, 0x50,
0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6d, 0x70,
0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03,
0x52, 0x0c, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1d,
0x0a, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x09, 0x20, 0x01,
0x28, 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a,
0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28,
0x09, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x54, 0x0a, 0x1a,
0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62,
0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x22,
0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02,
0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x49, 0x44, 0x22, 0x56, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74,
0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e,
0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d,
0x65, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f,
0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c,
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x86, 0x02, 0x0a, 0x13, 0x4c,
0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01,
0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52,
0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x49, 0x44,
0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x73, 0x12,
0x3d, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0e, 0x32,
0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69,
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f,
0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x18,
0x0a, 0x07, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52,
0x07, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x67,
0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x70, 0x72,
0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x6c,
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03,
0x28, 0x09, 0x52, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61,
0x6d, 0x65, 0x73, 0x2a, 0x45, 0x0a, 0x09, 0x52, 0x61, 0x74, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65,
0x12, 0x0b, 0x0a, 0x07, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x10, 0x00, 0x12, 0x0c, 0x0a,
0x08, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x43,
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x50,
0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x2a, 0xb9, 0x01, 0x0a, 0x08, 0x52,
0x61, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f,
0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x44, 0x44,
0x4c, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08,
0x44, 0x44, 0x4c, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x44,
0x4c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x03, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43,
0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x44,
0x4d, 0x4c, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d,
0x4c, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b, 0x44, 0x4d, 0x4c,
0x42, 0x75, 0x6c, 0x6b, 0x4c, 0x6f, 0x61, 0x64, 0x10, 0x07, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x51,
0x4c, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x10, 0x08, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x51, 0x4c,
0x51, 0x75, 0x65, 0x72, 0x79, 0x10, 0x09, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, 0x55, 0x70,
0x73, 0x65, 0x72, 0x74, 0x10, 0x0a, 0x2a, 0x81, 0x01, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72,
0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e,
0x65, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01,
0x12, 0x10, 0x0a, 0x0c, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67,
0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10,
0x03, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a,
0x09, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d,
0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x12,
0x09, 0x0a, 0x05, 0x53, 0x74, 0x61, 0x74, 0x73, 0x10, 0x07, 0x42, 0x32, 0x5a, 0x30, 0x67, 0x69,
0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d,
0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x62, 0x06,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (

View File

@ -35,6 +35,7 @@ enum MessageType {
DropPartition = 8;
ManualFlush = 9;
CreateSegment = 10;
Import = 11;
// begin transaction message is only used for transaction, once a begin
// transaction message is received, all messages combined with the
// transaction message cannot be consumed until a CommitTxn message
@ -204,6 +205,8 @@ message RollbackTxnMessageHeader {}
// Just do nothing now.
message TxnMessageHeader {}
message ImportMessageHeader {}
///
/// Message Extra Response
/// Used to add extra information when response to the client.

View File

@ -35,6 +35,7 @@ const (
MessageType_DropPartition MessageType = 8
MessageType_ManualFlush MessageType = 9
MessageType_CreateSegment MessageType = 10
MessageType_Import MessageType = 11
// begin transaction message is only used for transaction, once a begin
// transaction message is received, all messages combined with the
// transaction message cannot be consumed until a CommitTxn message
@ -72,6 +73,7 @@ var (
8: "DropPartition",
9: "ManualFlush",
10: "CreateSegment",
11: "Import",
900: "BeginTxn",
901: "CommitTxn",
902: "RollbackTxn",
@ -89,6 +91,7 @@ var (
"DropPartition": 8,
"ManualFlush": 9,
"CreateSegment": 10,
"Import": 11,
"BeginTxn": 900,
"CommitTxn": 901,
"RollbackTxn": 902,
@ -1574,6 +1577,44 @@ func (*TxnMessageHeader) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{26}
}
type ImportMessageHeader struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
}
func (x *ImportMessageHeader) Reset() {
*x = ImportMessageHeader{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[27]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ImportMessageHeader) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ImportMessageHeader) ProtoMessage() {}
func (x *ImportMessageHeader) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[27]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use ImportMessageHeader.ProtoReflect.Descriptor instead.
func (*ImportMessageHeader) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{27}
}
// ManualFlushExtraResponse is the extra response of manual flush message.
type ManualFlushExtraResponse struct {
state protoimpl.MessageState
@ -1586,7 +1627,7 @@ type ManualFlushExtraResponse struct {
func (x *ManualFlushExtraResponse) Reset() {
*x = ManualFlushExtraResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[27]
mi := &file_messages_proto_msgTypes[28]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -1599,7 +1640,7 @@ func (x *ManualFlushExtraResponse) String() string {
func (*ManualFlushExtraResponse) ProtoMessage() {}
func (x *ManualFlushExtraResponse) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[27]
mi := &file_messages_proto_msgTypes[28]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -1612,7 +1653,7 @@ func (x *ManualFlushExtraResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use ManualFlushExtraResponse.ProtoReflect.Descriptor instead.
func (*ManualFlushExtraResponse) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{27}
return file_messages_proto_rawDescGZIP(), []int{28}
}
func (x *ManualFlushExtraResponse) GetSegmentIds() []int64 {
@ -1640,7 +1681,7 @@ type TxnContext struct {
func (x *TxnContext) Reset() {
*x = TxnContext{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[28]
mi := &file_messages_proto_msgTypes[29]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -1653,7 +1694,7 @@ func (x *TxnContext) String() string {
func (*TxnContext) ProtoMessage() {}
func (x *TxnContext) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[28]
mi := &file_messages_proto_msgTypes[29]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -1666,7 +1707,7 @@ func (x *TxnContext) ProtoReflect() protoreflect.Message {
// Deprecated: Use TxnContext.ProtoReflect.Descriptor instead.
func (*TxnContext) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{28}
return file_messages_proto_rawDescGZIP(), []int{29}
}
func (x *TxnContext) GetTxnId() int64 {
@ -1696,7 +1737,7 @@ type RMQMessageLayout struct {
func (x *RMQMessageLayout) Reset() {
*x = RMQMessageLayout{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[29]
mi := &file_messages_proto_msgTypes[30]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -1709,7 +1750,7 @@ func (x *RMQMessageLayout) String() string {
func (*RMQMessageLayout) ProtoMessage() {}
func (x *RMQMessageLayout) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[29]
mi := &file_messages_proto_msgTypes[30]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -1722,7 +1763,7 @@ func (x *RMQMessageLayout) ProtoReflect() protoreflect.Message {
// Deprecated: Use RMQMessageLayout.ProtoReflect.Descriptor instead.
func (*RMQMessageLayout) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{29}
return file_messages_proto_rawDescGZIP(), []int{30}
}
func (x *RMQMessageLayout) GetPayload() []byte {
@ -1753,7 +1794,7 @@ type BroadcastHeader struct {
func (x *BroadcastHeader) Reset() {
*x = BroadcastHeader{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[30]
mi := &file_messages_proto_msgTypes[31]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -1766,7 +1807,7 @@ func (x *BroadcastHeader) String() string {
func (*BroadcastHeader) ProtoMessage() {}
func (x *BroadcastHeader) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[30]
mi := &file_messages_proto_msgTypes[31]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -1779,7 +1820,7 @@ func (x *BroadcastHeader) ProtoReflect() protoreflect.Message {
// Deprecated: Use BroadcastHeader.ProtoReflect.Descriptor instead.
func (*BroadcastHeader) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{30}
return file_messages_proto_rawDescGZIP(), []int{31}
}
func (x *BroadcastHeader) GetBroadcastId() uint64 {
@ -1818,7 +1859,7 @@ type ResourceKey struct {
func (x *ResourceKey) Reset() {
*x = ResourceKey{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[31]
mi := &file_messages_proto_msgTypes[32]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -1831,7 +1872,7 @@ func (x *ResourceKey) String() string {
func (*ResourceKey) ProtoMessage() {}
func (x *ResourceKey) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[31]
mi := &file_messages_proto_msgTypes[32]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -1844,7 +1885,7 @@ func (x *ResourceKey) ProtoReflect() protoreflect.Message {
// Deprecated: Use ResourceKey.ProtoReflect.Descriptor instead.
func (*ResourceKey) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{31}
return file_messages_proto_rawDescGZIP(), []int{32}
}
func (x *ResourceKey) GetDomain() ResourceDomain {
@ -1877,7 +1918,7 @@ type BroadcastEvent struct {
func (x *BroadcastEvent) Reset() {
*x = BroadcastEvent{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[32]
mi := &file_messages_proto_msgTypes[33]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -1890,7 +1931,7 @@ func (x *BroadcastEvent) String() string {
func (*BroadcastEvent) ProtoMessage() {}
func (x *BroadcastEvent) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[32]
mi := &file_messages_proto_msgTypes[33]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -1903,7 +1944,7 @@ func (x *BroadcastEvent) ProtoReflect() protoreflect.Message {
// Deprecated: Use BroadcastEvent.ProtoReflect.Descriptor instead.
func (*BroadcastEvent) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{32}
return file_messages_proto_rawDescGZIP(), []int{33}
}
func (m *BroadcastEvent) GetEvent() isBroadcastEvent_Event {
@ -1955,7 +1996,7 @@ type BroadcastResourceKeyAckAll struct {
func (x *BroadcastResourceKeyAckAll) Reset() {
*x = BroadcastResourceKeyAckAll{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[33]
mi := &file_messages_proto_msgTypes[34]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -1968,7 +2009,7 @@ func (x *BroadcastResourceKeyAckAll) String() string {
func (*BroadcastResourceKeyAckAll) ProtoMessage() {}
func (x *BroadcastResourceKeyAckAll) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[33]
mi := &file_messages_proto_msgTypes[34]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -1981,7 +2022,7 @@ func (x *BroadcastResourceKeyAckAll) ProtoReflect() protoreflect.Message {
// Deprecated: Use BroadcastResourceKeyAckAll.ProtoReflect.Descriptor instead.
func (*BroadcastResourceKeyAckAll) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{33}
return file_messages_proto_rawDescGZIP(), []int{34}
}
func (x *BroadcastResourceKeyAckAll) GetResourceKey() *ResourceKey {
@ -2003,7 +2044,7 @@ type BroadcastResourceKeyAckOne struct {
func (x *BroadcastResourceKeyAckOne) Reset() {
*x = BroadcastResourceKeyAckOne{}
if protoimpl.UnsafeEnabled {
mi := &file_messages_proto_msgTypes[34]
mi := &file_messages_proto_msgTypes[35]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -2016,7 +2057,7 @@ func (x *BroadcastResourceKeyAckOne) String() string {
func (*BroadcastResourceKeyAckOne) ProtoMessage() {}
func (x *BroadcastResourceKeyAckOne) ProtoReflect() protoreflect.Message {
mi := &file_messages_proto_msgTypes[34]
mi := &file_messages_proto_msgTypes[35]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -2029,7 +2070,7 @@ func (x *BroadcastResourceKeyAckOne) ProtoReflect() protoreflect.Message {
// Deprecated: Use BroadcastResourceKeyAckOne.ProtoReflect.Descriptor instead.
func (*BroadcastResourceKeyAckOne) Descriptor() ([]byte, []int) {
return file_messages_proto_rawDescGZIP(), []int{34}
return file_messages_proto_rawDescGZIP(), []int{35}
}
func (x *BroadcastResourceKeyAckOne) GetResourceKey() *ResourceKey {
@ -2176,107 +2217,109 @@ var file_messages_proto_rawDesc = []byte{
0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x1a, 0x0a, 0x18, 0x52, 0x6f,
0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x12, 0x0a, 0x10, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73,
0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 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, 0xe5, 0x01, 0x0a, 0x0e, 0x42, 0x72, 0x6f,
0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x64, 0x0a, 0x14, 0x72,
0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f,
0x61, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
0x73, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75,
0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x48, 0x00, 0x52, 0x11,
0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c,
0x6c, 0x12, 0x64, 0x0a, 0x14, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65,
0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f, 0x6f, 0x6e, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d,
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73,
0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f,
0x6e, 0x65, 0x48, 0x00, 0x52, 0x11, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65,
0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74,
0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x12, 0x45,
0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01,
0x20, 0x01, 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, 0x0b, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72,
0x63, 0x65, 0x4b, 0x65, 0x79, 0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61,
0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x15, 0x0a, 0x13, 0x49, 0x6d,
0x70, 0x6f, 0x72, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65,
0x72, 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,
0xe5, 0x01, 0x0a, 0x0e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x45, 0x76, 0x65,
0x6e, 0x74, 0x12, 0x64, 0x0a, 0x14, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b,
0x65, 0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
0x32, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61,
0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b,
0x4f, 0x6e, 0x65, 0x12, 0x45, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f,
0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 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, 0x0b, 0x72,
0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x2a, 0xfc, 0x01, 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, 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, 0x82, 0x01, 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, 0x0c, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x42, 0x65, 0x67,
0x69, 0x6e, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69,
0x67, 0x68, 0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f,
0x6d, 0x6d, 0x69, 0x74, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d,
0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f,
0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x54,
0x78, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0x06, 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, 0x32, 0x5a, 0x30,
0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x70, 0x62,
0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x41, 0x6c, 0x6c, 0x48, 0x00, 0x52, 0x11, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b,
0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x12, 0x64, 0x0a, 0x14, 0x72, 0x65, 0x73, 0x6f,
0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f, 0x6f, 0x6e, 0x65,
0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x42,
0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x48, 0x00, 0x52, 0x11, 0x72, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x42, 0x07,
0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64,
0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41,
0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x12, 0x45, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63,
0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 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,
0x0b, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x22, 0x63, 0x0a, 0x1a,
0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63,
0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x12, 0x45, 0x0a, 0x0c, 0x72, 0x65,
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 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, 0x0b, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65,
0x79, 0x2a, 0x88, 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, 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, 0x82, 0x01, 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, 0x0c, 0x0a, 0x08, 0x54, 0x78, 0x6e,
0x42, 0x65, 0x67, 0x69, 0x6e, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x49, 0x6e,
0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x4f,
0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x78, 0x6e,
0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x54,
0x78, 0x6e, 0x4f, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x10, 0x05, 0x12, 0x11,
0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10,
0x06, 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,
0x32, 0x5a, 0x30, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69,
0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70,
0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65,
0x73, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@ -2292,7 +2335,7 @@ func file_messages_proto_rawDescGZIP() []byte {
}
var file_messages_proto_enumTypes = make([]protoimpl.EnumInfo, 3)
var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 38)
var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 39)
var file_messages_proto_goTypes = []interface{}{
(MessageType)(0), // 0: milvus.proto.messages.MessageType
(TxnState)(0), // 1: milvus.proto.messages.TxnState
@ -2324,33 +2367,34 @@ var file_messages_proto_goTypes = []interface{}{
(*CommitTxnMessageHeader)(nil), // 27: milvus.proto.messages.CommitTxnMessageHeader
(*RollbackTxnMessageHeader)(nil), // 28: milvus.proto.messages.RollbackTxnMessageHeader
(*TxnMessageHeader)(nil), // 29: milvus.proto.messages.TxnMessageHeader
(*ManualFlushExtraResponse)(nil), // 30: milvus.proto.messages.ManualFlushExtraResponse
(*TxnContext)(nil), // 31: milvus.proto.messages.TxnContext
(*RMQMessageLayout)(nil), // 32: milvus.proto.messages.RMQMessageLayout
(*BroadcastHeader)(nil), // 33: milvus.proto.messages.BroadcastHeader
(*ResourceKey)(nil), // 34: milvus.proto.messages.ResourceKey
(*BroadcastEvent)(nil), // 35: milvus.proto.messages.BroadcastEvent
(*BroadcastResourceKeyAckAll)(nil), // 36: milvus.proto.messages.BroadcastResourceKeyAckAll
(*BroadcastResourceKeyAckOne)(nil), // 37: milvus.proto.messages.BroadcastResourceKeyAckOne
nil, // 38: milvus.proto.messages.Message.PropertiesEntry
nil, // 39: milvus.proto.messages.ImmutableMessage.PropertiesEntry
nil, // 40: milvus.proto.messages.RMQMessageLayout.PropertiesEntry
(*ImportMessageHeader)(nil), // 30: milvus.proto.messages.ImportMessageHeader
(*ManualFlushExtraResponse)(nil), // 31: milvus.proto.messages.ManualFlushExtraResponse
(*TxnContext)(nil), // 32: milvus.proto.messages.TxnContext
(*RMQMessageLayout)(nil), // 33: milvus.proto.messages.RMQMessageLayout
(*BroadcastHeader)(nil), // 34: milvus.proto.messages.BroadcastHeader
(*ResourceKey)(nil), // 35: milvus.proto.messages.ResourceKey
(*BroadcastEvent)(nil), // 36: milvus.proto.messages.BroadcastEvent
(*BroadcastResourceKeyAckAll)(nil), // 37: milvus.proto.messages.BroadcastResourceKeyAckAll
(*BroadcastResourceKeyAckOne)(nil), // 38: milvus.proto.messages.BroadcastResourceKeyAckOne
nil, // 39: milvus.proto.messages.Message.PropertiesEntry
nil, // 40: milvus.proto.messages.ImmutableMessage.PropertiesEntry
nil, // 41: milvus.proto.messages.RMQMessageLayout.PropertiesEntry
}
var file_messages_proto_depIdxs = []int32{
38, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry
39, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry
3, // 1: milvus.proto.messages.ImmutableMessage.id:type_name -> milvus.proto.messages.MessageID
39, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry
40, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry
9, // 3: milvus.proto.messages.CreateSegmentMessageBody.segments:type_name -> milvus.proto.messages.CreateSegmentInfo
4, // 4: milvus.proto.messages.TxnMessageBody.messages:type_name -> milvus.proto.messages.Message
16, // 5: milvus.proto.messages.InsertMessageHeader.partitions:type_name -> milvus.proto.messages.PartitionSegmentAssignment
17, // 6: milvus.proto.messages.PartitionSegmentAssignment.segment_assignment:type_name -> milvus.proto.messages.SegmentAssignment
40, // 7: milvus.proto.messages.RMQMessageLayout.properties:type_name -> milvus.proto.messages.RMQMessageLayout.PropertiesEntry
34, // 8: milvus.proto.messages.BroadcastHeader.Resource_keys:type_name -> milvus.proto.messages.ResourceKey
41, // 7: milvus.proto.messages.RMQMessageLayout.properties:type_name -> milvus.proto.messages.RMQMessageLayout.PropertiesEntry
35, // 8: milvus.proto.messages.BroadcastHeader.Resource_keys:type_name -> milvus.proto.messages.ResourceKey
2, // 9: milvus.proto.messages.ResourceKey.domain:type_name -> milvus.proto.messages.ResourceDomain
36, // 10: milvus.proto.messages.BroadcastEvent.resource_key_ack_all:type_name -> milvus.proto.messages.BroadcastResourceKeyAckAll
37, // 11: milvus.proto.messages.BroadcastEvent.resource_key_ack_one:type_name -> milvus.proto.messages.BroadcastResourceKeyAckOne
34, // 12: milvus.proto.messages.BroadcastResourceKeyAckAll.resource_key:type_name -> milvus.proto.messages.ResourceKey
34, // 13: milvus.proto.messages.BroadcastResourceKeyAckOne.resource_key:type_name -> milvus.proto.messages.ResourceKey
37, // 10: milvus.proto.messages.BroadcastEvent.resource_key_ack_all:type_name -> milvus.proto.messages.BroadcastResourceKeyAckAll
38, // 11: milvus.proto.messages.BroadcastEvent.resource_key_ack_one:type_name -> milvus.proto.messages.BroadcastResourceKeyAckOne
35, // 12: milvus.proto.messages.BroadcastResourceKeyAckAll.resource_key:type_name -> milvus.proto.messages.ResourceKey
35, // 13: milvus.proto.messages.BroadcastResourceKeyAckOne.resource_key:type_name -> milvus.proto.messages.ResourceKey
14, // [14:14] is the sub-list for method output_type
14, // [14:14] is the sub-list for method input_type
14, // [14:14] is the sub-list for extension type_name
@ -2689,7 +2733,7 @@ func file_messages_proto_init() {
}
}
file_messages_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ManualFlushExtraResponse); i {
switch v := v.(*ImportMessageHeader); i {
case 0:
return &v.state
case 1:
@ -2701,7 +2745,7 @@ func file_messages_proto_init() {
}
}
file_messages_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*TxnContext); i {
switch v := v.(*ManualFlushExtraResponse); i {
case 0:
return &v.state
case 1:
@ -2713,7 +2757,7 @@ func file_messages_proto_init() {
}
}
file_messages_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*RMQMessageLayout); i {
switch v := v.(*TxnContext); i {
case 0:
return &v.state
case 1:
@ -2725,7 +2769,7 @@ func file_messages_proto_init() {
}
}
file_messages_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BroadcastHeader); i {
switch v := v.(*RMQMessageLayout); i {
case 0:
return &v.state
case 1:
@ -2737,7 +2781,7 @@ func file_messages_proto_init() {
}
}
file_messages_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ResourceKey); i {
switch v := v.(*BroadcastHeader); i {
case 0:
return &v.state
case 1:
@ -2749,7 +2793,7 @@ func file_messages_proto_init() {
}
}
file_messages_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BroadcastEvent); i {
switch v := v.(*ResourceKey); i {
case 0:
return &v.state
case 1:
@ -2761,7 +2805,7 @@ func file_messages_proto_init() {
}
}
file_messages_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BroadcastResourceKeyAckAll); i {
switch v := v.(*BroadcastEvent); i {
case 0:
return &v.state
case 1:
@ -2773,6 +2817,18 @@ func file_messages_proto_init() {
}
}
file_messages_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BroadcastResourceKeyAckAll); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_messages_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*BroadcastResourceKeyAckOne); i {
case 0:
return &v.state
@ -2785,7 +2841,7 @@ func file_messages_proto_init() {
}
}
}
file_messages_proto_msgTypes[32].OneofWrappers = []interface{}{
file_messages_proto_msgTypes[33].OneofWrappers = []interface{}{
(*BroadcastEvent_ResourceKeyAckAll)(nil),
(*BroadcastEvent_ResourceKeyAckOne)(nil),
}
@ -2795,7 +2851,7 @@ func file_messages_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_messages_proto_rawDesc,
NumEnums: 3,
NumMessages: 38,
NumMessages: 39,
NumExtensions: 0,
NumServices: 0,
},

View File

@ -175,6 +175,12 @@ func recoverMessageFromHeader(tsMsg msgstream.TsMsg, msg message.ImmutableMessag
return nil, errors.Wrap(err, "Failed to convert message to delete message")
}
return recoverDeleteMsgFromHeader(tsMsg.(*msgstream.DeleteMsg), deleteMessage.Header(), msg.TimeTick())
case message.MessageTypeImport:
importMessage, err := message.AsImmutableImportMessageV1(msg)
if err != nil {
return nil, errors.Wrap(err, "Failed to convert message to import message")
}
return recoverImportMsgFromHeader(tsMsg.(*msgstream.ImportMsg), importMessage.Header(), msg.TimeTick())
default:
return tsMsg, nil
}
@ -220,3 +226,8 @@ func recoverDeleteMsgFromHeader(deleteMsg *msgstream.DeleteMsg, header *message.
deleteMsg.Timestamps = timestamps
return deleteMsg, nil
}
func recoverImportMsgFromHeader(importMsg *msgstream.ImportMsg, header *message.ImportMessageHeader, timetick uint64) (msgstream.TsMsg, error) {
importMsg.Base.Timestamp = timetick
return importMsg, nil
}

View File

@ -16,6 +16,7 @@ var messageTypeToCommonpbMsgType = map[message.MessageType]commonpb.MsgType{
message.MessageTypeDropCollection: commonpb.MsgType_DropCollection,
message.MessageTypeCreatePartition: commonpb.MsgType_CreatePartition,
message.MessageTypeDropPartition: commonpb.MsgType_DropPartition,
message.MessageTypeImport: commonpb.MsgType_Import,
}
// MustGetCommonpbMsgTypeFromMessageType returns the commonpb.MsgType from message.MessageType.

View File

@ -60,6 +60,7 @@ var (
NewDropCollectionMessageBuilderV1 = createNewMessageBuilderV1[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest]()
NewCreatePartitionMessageBuilderV1 = createNewMessageBuilderV1[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest]()
NewDropPartitionMessageBuilderV1 = createNewMessageBuilderV1[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest]()
NewImportMessageBuilderV1 = createNewMessageBuilderV1[*ImportMessageHeader, *msgpb.ImportMsg]()
NewCreateSegmentMessageBuilderV2 = createNewMessageBuilderV2[*CreateSegmentMessageHeader, *CreateSegmentMessageBody]()
NewFlushMessageBuilderV2 = createNewMessageBuilderV2[*FlushMessageHeader, *FlushMessageBody]()
NewManualFlushMessageBuilderV2 = createNewMessageBuilderV2[*ManualFlushMessageHeader, *ManualFlushMessageBody]()

View File

@ -24,6 +24,7 @@ const (
MessageTypeBeginTxn MessageType = MessageType(messagespb.MessageType_BeginTxn)
MessageTypeCommitTxn MessageType = MessageType(messagespb.MessageType_CommitTxn)
MessageTypeRollbackTxn MessageType = MessageType(messagespb.MessageType_RollbackTxn)
MessageTypeImport MessageType = MessageType(messagespb.MessageType_Import)
)
var messageTypeName = map[MessageType]string{
@ -42,6 +43,7 @@ var messageTypeName = map[MessageType]string{
MessageTypeBeginTxn: "BEGIN_TXN",
MessageTypeCommitTxn: "COMMIT_TXN",
MessageTypeRollbackTxn: "ROLLBACK_TXN",
MessageTypeImport: "IMPORT",
}
// String implements fmt.Stringer interface.

View File

@ -28,6 +28,7 @@ type (
CommitTxnMessageHeader = messagespb.CommitTxnMessageHeader
RollbackTxnMessageHeader = messagespb.RollbackTxnMessageHeader
TxnMessageHeader = messagespb.TxnMessageHeader
ImportMessageHeader = messagespb.ImportMessageHeader
)
type (
@ -60,6 +61,7 @@ var messageTypeMap = map[reflect.Type]MessageType{
reflect.TypeOf(&CommitTxnMessageHeader{}): MessageTypeCommitTxn,
reflect.TypeOf(&RollbackTxnMessageHeader{}): MessageTypeRollbackTxn,
reflect.TypeOf(&TxnMessageHeader{}): MessageTypeTxn,
reflect.TypeOf(&ImportMessageHeader{}): MessageTypeImport,
}
// A system preserved message, should not allowed to provide outside of the streaming system.
@ -80,6 +82,7 @@ type (
MutableDropCollectionMessageV1 = specializedMutableMessage[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest]
MutableCreatePartitionMessageV1 = specializedMutableMessage[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest]
MutableDropPartitionMessageV1 = specializedMutableMessage[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest]
MutableImportMessageV1 = specializedMutableMessage[*ImportMessageHeader, *msgpb.ImportMsg]
MutableCreateSegmentMessageV2 = specializedMutableMessage[*CreateSegmentMessageHeader, *CreateSegmentMessageBody]
MutableFlushMessageV2 = specializedMutableMessage[*FlushMessageHeader, *FlushMessageBody]
MutableBeginTxnMessageV2 = specializedMutableMessage[*BeginTxnMessageHeader, *BeginTxnMessageBody]
@ -93,6 +96,7 @@ type (
ImmutableDropCollectionMessageV1 = specializedImmutableMessage[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest]
ImmutableCreatePartitionMessageV1 = specializedImmutableMessage[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest]
ImmutableDropPartitionMessageV1 = specializedImmutableMessage[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest]
ImmutableImportMessageV1 = specializedImmutableMessage[*ImportMessageHeader, *msgpb.ImportMsg]
ImmutableCreateSegmentMessageV2 = specializedImmutableMessage[*CreateSegmentMessageHeader, *CreateSegmentMessageBody]
ImmutableFlushMessageV2 = specializedImmutableMessage[*FlushMessageHeader, *FlushMessageBody]
ImmutableManualFlushMessageV2 = specializedImmutableMessage[*ManualFlushMessageHeader, *ManualFlushMessageBody]
@ -110,6 +114,7 @@ var (
AsMutableDropCollectionMessageV1 = asSpecializedMutableMessage[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest]
AsMutableCreatePartitionMessageV1 = asSpecializedMutableMessage[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest]
AsMutableDropPartitionMessageV1 = asSpecializedMutableMessage[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest]
AsMutableImportMessageV1 = asSpecializedMutableMessage[*ImportMessageHeader, *msgpb.ImportMsg]
AsMutableCreateSegmentMessageV2 = asSpecializedMutableMessage[*CreateSegmentMessageHeader, *CreateSegmentMessageBody]
AsMutableFlushMessageV2 = asSpecializedMutableMessage[*FlushMessageHeader, *FlushMessageBody]
AsMutableManualFlushMessageV2 = asSpecializedMutableMessage[*ManualFlushMessageHeader, *ManualFlushMessageBody]
@ -124,6 +129,7 @@ var (
AsImmutableDropCollectionMessageV1 = asSpecializedImmutableMessage[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest]
AsImmutableCreatePartitionMessageV1 = asSpecializedImmutableMessage[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest]
AsImmutableDropPartitionMessageV1 = asSpecializedImmutableMessage[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest]
AsImmutableImportMessageV1 = asSpecializedImmutableMessage[*ImportMessageHeader, *msgpb.ImportMsg]
AsImmutableCreateSegmentMessageV2 = asSpecializedImmutableMessage[*CreateSegmentMessageHeader, *CreateSegmentMessageBody]
AsImmutableFlushMessageV2 = asSpecializedImmutableMessage[*FlushMessageHeader, *FlushMessageBody]
AsImmutableManualFlushMessageV2 = asSpecializedImmutableMessage[*ManualFlushMessageHeader, *ManualFlushMessageBody]
@ -143,7 +149,7 @@ var (
// Return nil, nil if the message is not the target specialized message.
// 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 MutableMessage) (specializedMutableMessage[H, B], error) {
func asSpecializedMutableMessage[H proto.Message, B proto.Message](msg BasicMessage) (specializedMutableMessage[H, B], error) {
underlying := msg.(*messageImpl)
var header H

View File

@ -40,6 +40,8 @@ for d in $(go list ./tests/integration/...); do
echo "running coordrecovery"
# simplified command to speed up coord init test since it is large.
$TEST_CMD -tags dynamic,test -v -coverprofile=profile.out -covermode=atomic "$d" -caseTimeout=20m -timeout=30m
elif [[ $d == *"import"* ]]; then
go test -tags dynamic,test -v -coverprofile=profile.out -covermode=atomic "$d" -caseTimeout=20m -timeout=60m
else
$TEST_CMD -race -tags dynamic,test -v -coverpkg=./... -coverprofile=profile.out -covermode=atomic "$d" -caseTimeout=15m -timeout=30m
fi

View File

@ -230,7 +230,7 @@ func (s *CompactionSuite) TestL0Compaction() {
// drop collection
// status, err = c.Proxy.DropCollection(ctx, &milvuspb.DropCollectionRequest{
// CollectionName: collectionName,
// CollectionName: collectionName,
// })
// err = merr.CheckRPCCall(status, err)
// s.NoError(err)

View File

@ -47,7 +47,7 @@ func (s *BulkInsertSuite) testImportDynamicField() {
)
c := s.Cluster
ctx, cancel := context.WithTimeout(c.GetContext(), 120*time.Second)
ctx, cancel := context.WithTimeout(c.GetContext(), 240*time.Second)
defer cancel()
collectionName := "TestBulkInsert_B_" + funcutil.GenRandomStr()

View File

@ -76,7 +76,7 @@ func (s *BulkInsertSuite) run() {
)
c := s.Cluster
ctx, cancel := context.WithTimeout(c.GetContext(), 120*time.Second)
ctx, cancel := context.WithTimeout(c.GetContext(), 240*time.Second)
defer cancel()
collectionName := "TestBulkInsert" + funcutil.GenRandomStr()

View File

@ -46,7 +46,7 @@ func (s *BulkInsertSuite) TestImportWithPartitionKey() {
)
c := s.Cluster
ctx, cancel := context.WithTimeout(c.GetContext(), 120*time.Second)
ctx, cancel := context.WithTimeout(c.GetContext(), 240*time.Second)
defer cancel()
collectionName := "TestBulkInsert_WithPartitionKey_" + funcutil.GenRandomStr()