enhance: implement external collection update task with source change detection (#45690)

issue: https://github.com/milvus-io/milvus/issues/45691
Add persistent task management for external collections with automatic
detection of external_source and external_spec changes. When source
changes, the system aborts running tasks and creates new ones, ensuring
only one active task per collection. Tasks validate their source on
completion to prevent superseded tasks from committing results.

---------

Signed-off-by: sunby <sunbingyi1992@gmail.com>
This commit is contained in:
Bingyi Sun 2025-11-27 15:33:08 +08:00 committed by GitHub
parent 7c9a9c6f7e
commit b6532d3e44
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
28 changed files with 2575 additions and 1044 deletions

2
go.mod
View File

@ -21,7 +21,7 @@ require (
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.6.6-0.20251119054300-fcb3986f4af1
github.com/milvus-io/milvus-proto/go-api/v2 v2.6.6-0.20251119072500-4bd276fe335e
github.com/minio/minio-go/v7 v7.0.73
github.com/panjf2000/ants/v2 v2.11.3 // indirect
github.com/pingcap/log v1.1.1-0.20221015072633-39906604fb81 // indirect

4
go.sum
View File

@ -798,8 +798,8 @@ github.com/milvus-io/cgosymbolizer v0.0.0-20250318084424-114f4050c3a6 h1:YHMFI6L
github.com/milvus-io/cgosymbolizer v0.0.0-20250318084424-114f4050c3a6/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.6.6-0.20251119054300-fcb3986f4af1 h1:nSZoftB+vB285AwYAOoJnwxKPMhP7l0p+VurCJGG9ds=
github.com/milvus-io/milvus-proto/go-api/v2 v2.6.6-0.20251119054300-fcb3986f4af1/go.mod h1:/6UT4zZl6awVeXLeE7UGDWZvXj3IWkRsh3mqsn0DiAs=
github.com/milvus-io/milvus-proto/go-api/v2 v2.6.6-0.20251119072500-4bd276fe335e h1:NZx+z8JDXrLf/y+bIs360ypdPDM9AZTkkA0Fi1v3MWc=
github.com/milvus-io/milvus-proto/go-api/v2 v2.6.6-0.20251119072500-4bd276fe335e/go.mod h1:/6UT4zZl6awVeXLeE7UGDWZvXj3IWkRsh3mqsn0DiAs=
github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 h1:AMFGa4R4MiIpspGNG7Z948v4n35fFGB3RR3G/ry4FWs=
github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8/go.mod h1:mC1jAcsrzbxHt8iiaC+zU4b1ylILSosueou12R++wfY=
github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 h1:+n/aFZefKZp7spd8DFdX7uMikMLXX4oubIzJF4kv/wI=

View File

@ -18,6 +18,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/msgpb"
"github.com/milvus-io/milvus/internal/datacoord"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/kv/tikv"
@ -1235,3 +1236,8 @@ func (s *mixCoordImpl) RemoveFileResource(ctx context.Context, req *milvuspb.Rem
func (s *mixCoordImpl) ListFileResources(ctx context.Context, req *milvuspb.ListFileResourcesRequest) (*milvuspb.ListFileResourcesResponse, error) {
return s.datacoordServer.ListFileResources(ctx, req)
}
// CreateExternalCollection creates an external collection
func (s *mixCoordImpl) CreateExternalCollection(ctx context.Context, req *msgpb.CreateCollectionRequest) (*datapb.CreateExternalCollectionResponse, error) {
return s.datacoordServer.CreateExternalCollection(ctx, req)
}

View File

@ -0,0 +1,265 @@
// 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 datacoord
import (
"context"
"fmt"
"sync"
"time"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/datacoord/allocator"
"github.com/milvus-io/milvus/internal/datacoord/task"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
)
type ExternalCollectionInspector interface {
Start()
Stop()
SubmitUpdateTask(collectionID int64) error
}
var _ ExternalCollectionInspector = (*externalCollectionInspector)(nil)
type externalCollectionInspector struct {
ctx context.Context
cancel context.CancelFunc
loopWg sync.WaitGroup
mt *meta
scheduler task.GlobalScheduler
allocator allocator.Allocator
}
func newExternalCollectionInspector(ctx context.Context,
mt *meta,
scheduler task.GlobalScheduler,
allocator allocator.Allocator,
) *externalCollectionInspector {
ctx, cancel := context.WithCancel(ctx)
return &externalCollectionInspector{
ctx: ctx,
cancel: cancel,
loopWg: sync.WaitGroup{},
mt: mt,
scheduler: scheduler,
allocator: allocator,
}
}
func (i *externalCollectionInspector) Start() {
i.reloadFromMeta()
i.loopWg.Add(1)
go i.triggerUpdateTaskLoop()
}
func (i *externalCollectionInspector) Stop() {
i.cancel()
i.loopWg.Wait()
}
func (i *externalCollectionInspector) reloadFromMeta() {
tasks := i.mt.externalCollectionTaskMeta.GetAllTasks()
for _, t := range tasks {
if t.GetState() != indexpb.JobState_JobStateInit &&
t.GetState() != indexpb.JobState_JobStateRetry &&
t.GetState() != indexpb.JobState_JobStateInProgress {
continue
}
// Enqueue active tasks for processing
updateTask := newUpdateExternalCollectionTask(t, i.mt)
i.scheduler.Enqueue(updateTask)
}
}
func (i *externalCollectionInspector) triggerUpdateTaskLoop() {
log.Info("start external collection inspector loop...")
defer i.loopWg.Done()
ticker := time.NewTicker(Params.DataCoordCfg.TaskCheckInterval.GetAsDuration(time.Second))
defer ticker.Stop()
for {
select {
case <-i.ctx.Done():
log.Warn("DataCoord context done, exit external collection inspector loop...")
return
case <-ticker.C:
i.triggerUpdateTasks()
}
}
}
func (i *externalCollectionInspector) triggerUpdateTasks() {
collections := i.mt.GetCollections()
for _, collection := range collections {
if !i.isExternalCollection(collection) {
continue
}
// Check if we should trigger a task based on source changes
if shouldTrigger := i.shouldTriggerTask(collection); shouldTrigger {
if err := i.SubmitUpdateTask(collection.ID); err != nil {
log.Warn("failed to submit update task for external collection",
zap.Int64("collectionID", collection.ID),
zap.Error(err))
}
}
}
}
func (i *externalCollectionInspector) isExternalCollection(collection *collectionInfo) bool {
if collection.Schema == nil {
return false
}
return collection.Schema.GetExternalSource() != ""
}
// shouldTriggerTask checks if we should create/replace a task based on external source changes
// Returns shouldTrigger
func (i *externalCollectionInspector) shouldTriggerTask(collection *collectionInfo) bool {
externalSource := collection.Schema.GetExternalSource()
externalSpec := collection.Schema.GetExternalSpec()
existingTask := i.mt.externalCollectionTaskMeta.GetTaskByCollectionID(collection.ID)
if existingTask == nil {
// No task exists, create one
return true
}
taskSource := existingTask.GetExternalSource()
taskSpec := existingTask.GetExternalSpec()
// Check if source or spec changed
sourceChanged := externalSource != taskSource || externalSpec != taskSpec
switch existingTask.GetState() {
case indexpb.JobState_JobStateInit, indexpb.JobState_JobStateRetry, indexpb.JobState_JobStateInProgress:
// Task is running
if sourceChanged {
// Source changed while task running - abort and replace
log.Info("External source changed, replacing running task",
zap.Int64("collectionID", collection.ID),
zap.Int64("taskID", existingTask.GetTaskID()),
zap.String("oldSource", taskSource),
zap.String("newSource", externalSource))
// Abort old task from scheduler
i.scheduler.AbortAndRemoveTask(existingTask.GetTaskID())
// Drop from meta
i.mt.externalCollectionTaskMeta.DropTask(context.Background(), existingTask.GetTaskID())
return true
}
// Same source, task already running
return false
case indexpb.JobState_JobStateFinished:
// Task completed
if sourceChanged {
// Source changed after completion, create new task
log.Info("External source changed after completion, creating new task",
zap.Int64("collectionID", collection.ID),
zap.String("oldSource", taskSource),
zap.String("newSource", externalSource))
// Drop old completed task, create new one
i.mt.externalCollectionTaskMeta.DropTask(context.Background(), existingTask.GetTaskID())
return true
}
// Up to date
return false
case indexpb.JobState_JobStateFailed:
if sourceChanged {
// Source changed after failure - worth retrying with new source
log.Info("External source changed after failure, creating new task",
zap.Int64("collectionID", collection.ID),
zap.String("oldSource", taskSource),
zap.String("newSource", externalSource))
i.mt.externalCollectionTaskMeta.DropTask(context.Background(), existingTask.GetTaskID())
return true
}
// Same source failed - don't auto-retry
return false
default:
return false
}
}
func (i *externalCollectionInspector) SubmitUpdateTask(collectionID int64) error {
log := log.Ctx(i.ctx).With(zap.Int64("collectionID", collectionID))
// Get collection info to retrieve external source and spec
collection := i.mt.GetCollection(collectionID)
if collection == nil {
log.Warn("collection not found")
return fmt.Errorf("collection %d not found", collectionID)
}
// Allocate task ID
taskID, err := i.allocator.AllocID(context.Background())
if err != nil {
log.Warn("failed to allocate task ID", zap.Error(err))
return err
}
// Create task
t := &indexpb.UpdateExternalCollectionTask{
CollectionID: collectionID,
TaskID: taskID,
Version: 0,
NodeID: 0,
State: indexpb.JobState_JobStateInit,
FailReason: "",
ExternalSource: collection.Schema.GetExternalSource(),
ExternalSpec: collection.Schema.GetExternalSpec(),
}
// Add task to meta
if err = i.mt.externalCollectionTaskMeta.AddTask(t); err != nil {
if errors.Is(err, merr.ErrTaskDuplicate) {
log.Info("external collection update task already exists",
zap.Int64("collectionID", collectionID))
return nil
}
log.Warn("failed to add task to meta", zap.Error(err))
return err
}
// Create and enqueue task
updateTask := newUpdateExternalCollectionTask(t, i.mt)
i.scheduler.Enqueue(updateTask)
log.Info("external collection update task submitted",
zap.Int64("taskID", taskID),
zap.Int64("collectionID", collectionID),
zap.String("externalSource", collection.Schema.GetExternalSource()))
return nil
}

View File

@ -0,0 +1,221 @@
// 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 datacoord
import (
"context"
"fmt"
"strconv"
"go.uber.org/zap"
"google.golang.org/protobuf/proto"
"github.com/milvus-io/milvus/internal/metastore"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/util/lock"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/timerecord"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
type externalCollectionTaskMeta struct {
ctx context.Context
catalog metastore.DataCoordCatalog
keyLock *lock.KeyLock[UniqueID]
// taskID -> UpdateExternalCollectionTask
tasks *typeutil.ConcurrentMap[UniqueID, *indexpb.UpdateExternalCollectionTask]
// collectionID -> UpdateExternalCollectionTask
collectionID2Tasks *typeutil.ConcurrentMap[UniqueID, *indexpb.UpdateExternalCollectionTask]
}
func newExternalCollectionTaskMeta(ctx context.Context, catalog metastore.DataCoordCatalog) (*externalCollectionTaskMeta, error) {
ectm := &externalCollectionTaskMeta{
ctx: ctx,
catalog: catalog,
keyLock: lock.NewKeyLock[UniqueID](),
tasks: typeutil.NewConcurrentMap[UniqueID, *indexpb.UpdateExternalCollectionTask](),
collectionID2Tasks: typeutil.NewConcurrentMap[UniqueID, *indexpb.UpdateExternalCollectionTask](),
}
if err := ectm.reloadFromKV(); err != nil {
return nil, err
}
return ectm, nil
}
func (ectm *externalCollectionTaskMeta) reloadFromKV() error {
record := timerecord.NewTimeRecorder("externalCollectionTaskMeta-reloadFromKV")
tasks, err := ectm.catalog.ListUpdateExternalCollectionTasks(ectm.ctx)
if err != nil {
log.Error("externalCollectionTaskMeta reloadFromKV load tasks failed", zap.Error(err))
return err
}
for _, t := range tasks {
ectm.tasks.Insert(t.GetTaskID(), t)
ectm.collectionID2Tasks.Insert(t.GetCollectionID(), t)
}
log.Info("externalCollectionTaskMeta reloadFromKV done", zap.Duration("duration", record.ElapseSpan()))
return nil
}
func (ectm *externalCollectionTaskMeta) AddTask(t *indexpb.UpdateExternalCollectionTask) error {
ectm.keyLock.Lock(t.GetTaskID())
defer ectm.keyLock.Unlock(t.GetTaskID())
log.Ctx(ectm.ctx).Info("add update external collection task",
zap.Int64("taskID", t.GetTaskID()),
zap.Int64("collectionID", t.GetCollectionID()))
if _, ok := ectm.collectionID2Tasks.Get(t.GetCollectionID()); ok {
log.Warn("update external collection task already exists for collection",
zap.Int64("collectionID", t.GetCollectionID()))
return merr.WrapErrTaskDuplicate(strconv.FormatInt(t.GetCollectionID(), 10))
}
if err := ectm.catalog.SaveUpdateExternalCollectionTask(ectm.ctx, t); err != nil {
log.Warn("save update external collection task failed",
zap.Int64("taskID", t.GetTaskID()),
zap.Int64("collectionID", t.GetCollectionID()),
zap.Error(err))
return err
}
ectm.tasks.Insert(t.GetTaskID(), t)
ectm.collectionID2Tasks.Insert(t.GetCollectionID(), t)
log.Info("add update external collection task success",
zap.Int64("taskID", t.GetTaskID()),
zap.Int64("collectionID", t.GetCollectionID()))
return nil
}
func (ectm *externalCollectionTaskMeta) DropTask(ctx context.Context, taskID int64) error {
ectm.keyLock.Lock(taskID)
defer ectm.keyLock.Unlock(taskID)
log.Ctx(ctx).Info("drop update external collection task by taskID", zap.Int64("taskID", taskID))
t, ok := ectm.tasks.Get(taskID)
if !ok {
log.Info("remove update external collection task success, task already not exist", zap.Int64("taskID", taskID))
return nil
}
if err := ectm.catalog.DropUpdateExternalCollectionTask(ctx, taskID); err != nil {
log.Warn("drop update external collection task failed",
zap.Int64("taskID", taskID),
zap.Int64("collectionID", t.GetCollectionID()),
zap.Error(err))
return err
}
ectm.tasks.Remove(taskID)
ectm.collectionID2Tasks.Remove(t.GetCollectionID())
log.Info("remove update external collection task success", zap.Int64("taskID", taskID))
return nil
}
func (ectm *externalCollectionTaskMeta) UpdateVersion(taskID, nodeID int64) error {
ectm.keyLock.Lock(taskID)
defer ectm.keyLock.Unlock(taskID)
t, ok := ectm.tasks.Get(taskID)
if !ok {
return fmt.Errorf("task %d not found", taskID)
}
cloneT := proto.Clone(t).(*indexpb.UpdateExternalCollectionTask)
cloneT.Version++
cloneT.NodeID = nodeID
if err := ectm.catalog.SaveUpdateExternalCollectionTask(ectm.ctx, cloneT); err != nil {
log.Warn("update external collection task version failed",
zap.Int64("taskID", t.GetTaskID()),
zap.Int64("collectionID", t.GetCollectionID()),
zap.Int64("nodeID", nodeID),
zap.Error(err))
return err
}
ectm.tasks.Insert(taskID, cloneT)
ectm.collectionID2Tasks.Insert(t.GetCollectionID(), cloneT)
log.Info("update external collection task version success", zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID),
zap.Int64("newVersion", cloneT.GetVersion()))
return nil
}
func (ectm *externalCollectionTaskMeta) UpdateTaskState(taskID int64, state indexpb.JobState, failReason string) error {
ectm.keyLock.Lock(taskID)
defer ectm.keyLock.Unlock(taskID)
t, ok := ectm.tasks.Get(taskID)
if !ok {
return fmt.Errorf("task %d not found", taskID)
}
cloneT := proto.Clone(t).(*indexpb.UpdateExternalCollectionTask)
cloneT.State = state
cloneT.FailReason = failReason
if err := ectm.catalog.SaveUpdateExternalCollectionTask(ectm.ctx, cloneT); err != nil {
log.Warn("update external collection task state failed",
zap.Int64("taskID", t.GetTaskID()),
zap.Error(err))
return err
}
ectm.tasks.Insert(taskID, cloneT)
ectm.collectionID2Tasks.Insert(t.GetCollectionID(), cloneT)
return nil
}
func (ectm *externalCollectionTaskMeta) GetTask(taskID int64) *indexpb.UpdateExternalCollectionTask {
t, ok := ectm.tasks.Get(taskID)
if !ok {
return nil
}
return proto.Clone(t).(*indexpb.UpdateExternalCollectionTask)
}
func (ectm *externalCollectionTaskMeta) GetTaskState(taskID int64) indexpb.JobState {
t, ok := ectm.tasks.Get(taskID)
if !ok {
return indexpb.JobState_JobStateNone
}
return t.State
}
func (ectm *externalCollectionTaskMeta) GetTaskByCollectionID(collectionID int64) *indexpb.UpdateExternalCollectionTask {
t, ok := ectm.collectionID2Tasks.Get(collectionID)
if !ok {
return nil
}
return proto.Clone(t).(*indexpb.UpdateExternalCollectionTask)
}
func (ectm *externalCollectionTaskMeta) GetAllTasks() map[int64]*indexpb.UpdateExternalCollectionTask {
tasks := make(map[int64]*indexpb.UpdateExternalCollectionTask)
ectm.tasks.Range(func(taskID int64, task *indexpb.UpdateExternalCollectionTask) bool {
tasks[taskID] = proto.Clone(task).(*indexpb.UpdateExternalCollectionTask)
return true
})
return tasks
}

View File

@ -99,6 +99,7 @@ type meta struct {
partitionStatsMeta *partitionStatsMeta
compactionTaskMeta *compactionTaskMeta
statsTaskMeta *statsTaskMeta
externalCollectionTaskMeta *externalCollectionTaskMeta
// File Resource Meta
resourceMeta map[string]*model.FileResource
@ -183,6 +184,13 @@ func newMeta(ctx context.Context, catalog metastore.DataCoordCatalog, chunkManag
if err != nil {
return nil, err
}
// TODO: add external collection task meta
// ectm, err := newExternalCollectionTaskMeta(ctx, catalog)
// if err != nil {
// return nil, err
// }
mt := &meta{
ctx: ctx,
catalog: catalog,
@ -195,6 +203,7 @@ func newMeta(ctx context.Context, catalog metastore.DataCoordCatalog, chunkManag
partitionStatsMeta: psm,
compactionTaskMeta: ctm,
statsTaskMeta: stm,
// externalCollectionTaskMeta: ectm,
resourceMeta: make(map[string]*model.FileResource),
}
err = mt.reloadFromKV(ctx, broker)

View File

@ -28,6 +28,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/msgpb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/datacoord/allocator"
"github.com/milvus-io/milvus/internal/datacoord/broker"
@ -215,6 +216,12 @@ func (m *mockMixCoord) FlushAll(ctx context.Context, req *datapb.FlushAllRequest
panic("implement me")
}
func (m *mockMixCoord) CreateExternalCollection(ctx context.Context, req *msgpb.CreateCollectionRequest) (*datapb.CreateExternalCollectionResponse, error) {
return &datapb.CreateExternalCollectionResponse{
Status: merr.Success(),
}, nil
}
func newMockMixCoord() *mockMixCoord {
return &mockMixCoord{state: commonpb.StateCode_Healthy}
}

View File

@ -159,6 +159,7 @@ type Server struct {
statsInspector *statsInspector
indexInspector *indexInspector
analyzeInspector *analyzeInspector
externalCollectionInspector *externalCollectionInspector
globalScheduler task.GlobalScheduler
// manage ways that data coord access other coord
@ -321,6 +322,10 @@ func (s *Server) initDataCoord() error {
s.initStatsInspector()
log.Info("init statsJobManager done")
// TODO: enable external collection inspector
// s.initExternalCollectionInspector()
// log.Info("init external collection inspector done")
if err = s.initSegmentManager(); err != nil {
return err
}
@ -674,6 +679,12 @@ func (s *Server) initStatsInspector() {
}
}
func (s *Server) initExternalCollectionInspector() {
if s.externalCollectionInspector == nil {
s.externalCollectionInspector = newExternalCollectionInspector(s.ctx, s.meta, s.globalScheduler, s.allocator)
}
}
func (s *Server) initCompaction() {
cph := newCompactionInspector(s.meta, s.allocator, s.handler, s.globalScheduler, s.indexEngineVersionManager)
cph.loadMeta()
@ -749,6 +760,8 @@ func (s *Server) startTaskScheduler() {
s.statsInspector.Start()
s.indexInspector.Start()
s.analyzeInspector.Start()
// TODO: enable external collection inspector
// s.externalCollectionInspector.Start()
s.startCollectMetaMetrics(s.serverLoopCtx)
}
@ -1060,6 +1073,9 @@ func (s *Server) Stop() error {
if s.qnSessionWatcher != nil {
s.qnSessionWatcher.Stop()
}
// TODO: enable external collection inspector
// s.externalCollectionInspector.Stop()
// log.Info("datacoord external collection inspector stopped")
if s.session != nil {
s.session.Stop()

View File

@ -2098,3 +2098,48 @@ func (s *Server) ListFileResources(ctx context.Context, req *milvuspb.ListFileRe
Resources: fileResources,
}, nil
}
// CreateExternalCollection creates an external collection in datacoord
// This is a skeleton implementation - details to be filled in later
func (s *Server) CreateExternalCollection(ctx context.Context, req *msgpb.CreateCollectionRequest) (*datapb.CreateExternalCollectionResponse, error) {
log := log.Ctx(ctx).With(
zap.String("dbName", req.GetDbName()),
zap.String("collectionName", req.GetCollectionName()),
zap.Int64("dbID", req.GetDbID()),
zap.Int64("collectionID", req.GetCollectionID()))
log.Info("receive CreateExternalCollection request")
// Check if server is healthy
if err := merr.CheckHealthy(s.GetStateCode()); err != nil {
log.Warn("server is not healthy", zap.Error(err))
return &datapb.CreateExternalCollectionResponse{
Status: merr.Status(err),
}, nil
}
// Create collection info and add to meta
// This will make the collection visible to inspectors
// The collection schema already contains external_source and external_spec fields
collInfo := &collectionInfo{
ID: req.GetCollectionID(),
Schema: req.GetCollectionSchema(),
Partitions: req.GetPartitionIDs(),
Properties: make(map[string]string),
DatabaseID: req.GetDbID(),
DatabaseName: req.GetDbName(),
VChannelNames: req.GetVirtualChannelNames(),
}
s.meta.AddCollection(collInfo)
log.Info("CreateExternalCollection: collection added to meta",
zap.Int64("collectionID", req.GetCollectionID()),
zap.String("collectionName", req.GetCollectionName()),
zap.String("externalSource", req.GetCollectionSchema().GetExternalSource()),
zap.String("externalSpec", req.GetCollectionSchema().GetExternalSpec()))
return &datapb.CreateExternalCollectionResponse{
Status: merr.Success(),
}, nil
}

View File

@ -0,0 +1,149 @@
// 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 datacoord
import (
"context"
"fmt"
"time"
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/datacoord/session"
globalTask "github.com/milvus-io/milvus/internal/datacoord/task"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/taskcommon"
)
type updateExternalCollectionTask struct {
*indexpb.UpdateExternalCollectionTask
times *taskcommon.Times
meta *meta
}
var _ globalTask.Task = (*updateExternalCollectionTask)(nil)
func newUpdateExternalCollectionTask(t *indexpb.UpdateExternalCollectionTask, mt *meta) *updateExternalCollectionTask {
return &updateExternalCollectionTask{
UpdateExternalCollectionTask: t,
times: taskcommon.NewTimes(),
meta: mt,
}
}
func (t *updateExternalCollectionTask) GetTaskID() int64 {
return t.TaskID
}
func (t *updateExternalCollectionTask) GetTaskType() taskcommon.Type {
// Reuse Stats type for now, or we could add a new type
return taskcommon.Stats
}
func (t *updateExternalCollectionTask) GetTaskState() taskcommon.State {
return t.GetState()
}
func (t *updateExternalCollectionTask) GetTaskSlot() int64 {
// External collection tasks are lightweight, use 1 slot
return 1
}
func (t *updateExternalCollectionTask) SetTaskTime(timeType taskcommon.TimeType, time time.Time) {
t.times.SetTaskTime(timeType, time)
}
func (t *updateExternalCollectionTask) GetTaskTime(timeType taskcommon.TimeType) time.Time {
return timeType.GetTaskTime(t.times)
}
func (t *updateExternalCollectionTask) GetTaskVersion() int64 {
return t.GetVersion()
}
// validateSource checks if this task's external source matches the current collection source
// Returns error if task has been superseded
func (t *updateExternalCollectionTask) validateSource() error {
collection := t.meta.GetCollection(t.GetCollectionID())
if collection == nil {
return fmt.Errorf("collection %d not found", t.GetCollectionID())
}
currentSource := collection.Schema.GetExternalSource()
currentSpec := collection.Schema.GetExternalSpec()
taskSource := t.GetExternalSource()
taskSpec := t.GetExternalSpec()
if currentSource != taskSource || currentSpec != taskSpec {
return fmt.Errorf("task source mismatch: task source=%s/%s, current source=%s/%s (task has been superseded)",
taskSource, taskSpec, currentSource, currentSpec)
}
return nil
}
func (t *updateExternalCollectionTask) SetState(state indexpb.JobState, failReason string) {
// If transitioning to finished state, validate source first
if state == indexpb.JobState_JobStateFinished {
if err := t.validateSource(); err != nil {
log.Warn("Task source validation failed, marking as failed instead",
zap.Int64("taskID", t.GetTaskID()),
zap.Int64("collectionID", t.GetCollectionID()),
zap.Error(err))
t.State = indexpb.JobState_JobStateFailed
t.FailReason = fmt.Sprintf("source mismatch: %s", err.Error())
return
}
}
t.State = state
t.FailReason = failReason
}
func (t *updateExternalCollectionTask) CreateTaskOnWorker(nodeID int64, cluster session.Cluster) {
ctx := context.Background()
log := log.Ctx(ctx).With(
zap.Int64("taskID", t.GetTaskID()),
zap.Int64("collectionID", t.GetCollectionID()),
)
// For external collections, we just need to update metadata
// This is a placeholder for actual logic that would:
// 1. Query external storage for collection statistics
// 2. Update collection metadata in meta
// 3. Mark task as finished
log.Info("updating external collection metadata")
// TODO: Implement actual update logic
// For now, just mark as finished
t.SetState(indexpb.JobState_JobStateFinished, "")
log.Info("external collection metadata updated successfully")
}
func (t *updateExternalCollectionTask) QueryTaskOnWorker(cluster session.Cluster) {
// External collection tasks finish immediately, so query is a no-op
}
func (t *updateExternalCollectionTask) DropTaskOnWorker(cluster session.Cluster) {
// External collection tasks don't run on workers, so drop is a no-op
}

View File

@ -27,6 +27,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/msgpb"
"github.com/milvus-io/milvus/internal/distributed/utils"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/grpcclient"
@ -896,6 +897,17 @@ func (c *Client) FlushAll(ctx context.Context, req *datapb.FlushAllRequest, opts
})
}
func (c *Client) CreateExternalCollection(ctx context.Context, req *msgpb.CreateCollectionRequest, opts ...grpc.CallOption) (*datapb.CreateExternalCollectionResponse, error) {
req = typeutil.Clone(req)
commonpbutil.UpdateMsgBase(
req.GetBase(),
commonpbutil.FillMsgBaseFromClient(paramtable.GetNodeID(), commonpbutil.WithTargetID(c.grpcClient.GetNodeID())),
)
return wrapGrpcCall(ctx, c, func(client MixCoordClient) (*datapb.CreateExternalCollectionResponse, error) {
return client.CreateExternalCollection(ctx, req)
})
}
// AssignSegmentID applies allocations for specified Coolection/Partition and related Channel Name(Virtial Channel)
//
// ctx is the context to control request deadline and cancellation

View File

@ -31,6 +31,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/msgpb"
mixcoord "github.com/milvus-io/milvus/internal/coordinator"
mix "github.com/milvus-io/milvus/internal/distributed/mixcoord/client"
"github.com/milvus-io/milvus/internal/distributed/utils"
@ -729,6 +730,11 @@ func (s *Server) FlushAll(ctx context.Context, req *datapb.FlushAllRequest) (*da
return s.mixCoord.FlushAll(ctx, req)
}
// CreateExternalCollection creates an external collection
func (s *Server) CreateExternalCollection(ctx context.Context, req *msgpb.CreateCollectionRequest) (*datapb.CreateExternalCollectionResponse, error) {
return s.mixCoord.CreateExternalCollection(ctx, req)
}
// AssignSegmentID requests to allocate segment space for insert
func (s *Server) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error) {
return s.mixCoord.AssignSegmentID(ctx, req)

View File

@ -180,6 +180,10 @@ type DataCoordCatalog interface {
SaveStatsTask(ctx context.Context, task *indexpb.StatsTask) error
DropStatsTask(ctx context.Context, taskID typeutil.UniqueID) error
ListUpdateExternalCollectionTasks(ctx context.Context) ([]*indexpb.UpdateExternalCollectionTask, error)
SaveUpdateExternalCollectionTask(ctx context.Context, task *indexpb.UpdateExternalCollectionTask) error
DropUpdateExternalCollectionTask(ctx context.Context, taskID typeutil.UniqueID) error
// Analyzer Resource
SaveFileResource(ctx context.Context, resource *model.FileResource) error
RemoveFileResource(ctx context.Context, resourceID int64) error

View File

@ -33,6 +33,7 @@ const (
PartitionStatsInfoPrefix = MetaPrefix + "/partition-stats"
PartitionStatsCurrentVersionPrefix = MetaPrefix + "/current-partition-stats-version"
StatsTaskPrefix = MetaPrefix + "/stats-task"
UpdateExternalCollectionTaskPrefix = MetaPrefix + "/update-external-collection-task"
FileResourceMetaPrefix = MetaPrefix + "/file_resource"
NonRemoveFlagTomestone = "non-removed"

View File

@ -963,6 +963,45 @@ func (kc *Catalog) DropStatsTask(ctx context.Context, taskID typeutil.UniqueID)
return kc.MetaKv.Remove(ctx, key)
}
func (kc *Catalog) ListUpdateExternalCollectionTasks(ctx context.Context) ([]*indexpb.UpdateExternalCollectionTask, error) {
tasks := make([]*indexpb.UpdateExternalCollectionTask, 0)
applyFn := func(key []byte, value []byte) error {
task := &indexpb.UpdateExternalCollectionTask{}
err := proto.Unmarshal(value, task)
if err != nil {
return err
}
tasks = append(tasks, task)
return nil
}
err := kc.MetaKv.WalkWithPrefix(ctx, UpdateExternalCollectionTaskPrefix, kc.paginationSize, applyFn)
if err != nil {
return nil, err
}
return tasks, nil
}
func (kc *Catalog) SaveUpdateExternalCollectionTask(ctx context.Context, task *indexpb.UpdateExternalCollectionTask) error {
key := buildUpdateExternalCollectionTaskKey(task.TaskID)
value, err := proto.Marshal(task)
if err != nil {
return err
}
err = kc.MetaKv.Save(ctx, key, string(value))
if err != nil {
return err
}
return nil
}
func (kc *Catalog) DropUpdateExternalCollectionTask(ctx context.Context, taskID typeutil.UniqueID) error {
key := buildUpdateExternalCollectionTaskKey(taskID)
return kc.MetaKv.Remove(ctx, key)
}
func (kc *Catalog) SaveFileResource(ctx context.Context, resource *model.FileResource) error {
k := BuildFileResourceKey(resource.ID)
v, err := proto.Marshal(resource.Marshal())

View File

@ -352,3 +352,7 @@ func buildAnalyzeTaskKey(taskID int64) string {
func buildStatsTaskKey(taskID int64) string {
return fmt.Sprintf("%s/%d", StatsTaskPrefix, taskID)
}
func buildUpdateExternalCollectionTaskKey(taskID int64) string {
return fmt.Sprintf("%s/%d", UpdateExternalCollectionTaskPrefix, taskID)
}

View File

@ -991,6 +991,53 @@ func (_c *DataCoordCatalog_DropStatsTask_Call) RunAndReturn(run func(context.Con
return _c
}
// DropUpdateExternalCollectionTask provides a mock function with given fields: ctx, taskID
func (_m *DataCoordCatalog) DropUpdateExternalCollectionTask(ctx context.Context, taskID int64) error {
ret := _m.Called(ctx, taskID)
if len(ret) == 0 {
panic("no return value specified for DropUpdateExternalCollectionTask")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, int64) error); ok {
r0 = rf(ctx, taskID)
} else {
r0 = ret.Error(0)
}
return r0
}
// DataCoordCatalog_DropUpdateExternalCollectionTask_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'DropUpdateExternalCollectionTask'
type DataCoordCatalog_DropUpdateExternalCollectionTask_Call struct {
*mock.Call
}
// DropUpdateExternalCollectionTask is a helper method to define mock.On call
// - ctx context.Context
// - taskID int64
func (_e *DataCoordCatalog_Expecter) DropUpdateExternalCollectionTask(ctx interface{}, taskID interface{}) *DataCoordCatalog_DropUpdateExternalCollectionTask_Call {
return &DataCoordCatalog_DropUpdateExternalCollectionTask_Call{Call: _e.mock.On("DropUpdateExternalCollectionTask", ctx, taskID)}
}
func (_c *DataCoordCatalog_DropUpdateExternalCollectionTask_Call) Run(run func(ctx context.Context, taskID int64)) *DataCoordCatalog_DropUpdateExternalCollectionTask_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(int64))
})
return _c
}
func (_c *DataCoordCatalog_DropUpdateExternalCollectionTask_Call) Return(_a0 error) *DataCoordCatalog_DropUpdateExternalCollectionTask_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *DataCoordCatalog_DropUpdateExternalCollectionTask_Call) RunAndReturn(run func(context.Context, int64) error) *DataCoordCatalog_DropUpdateExternalCollectionTask_Call {
_c.Call.Return(run)
return _c
}
// GcConfirm provides a mock function with given fields: ctx, collectionID, partitionID
func (_m *DataCoordCatalog) GcConfirm(ctx context.Context, collectionID int64, partitionID int64) bool {
ret := _m.Called(ctx, collectionID, partitionID)
@ -1795,6 +1842,64 @@ func (_c *DataCoordCatalog_ListStatsTasks_Call) RunAndReturn(run func(context.Co
return _c
}
// ListUpdateExternalCollectionTasks provides a mock function with given fields: ctx
func (_m *DataCoordCatalog) ListUpdateExternalCollectionTasks(ctx context.Context) ([]*indexpb.UpdateExternalCollectionTask, error) {
ret := _m.Called(ctx)
if len(ret) == 0 {
panic("no return value specified for ListUpdateExternalCollectionTasks")
}
var r0 []*indexpb.UpdateExternalCollectionTask
var r1 error
if rf, ok := ret.Get(0).(func(context.Context) ([]*indexpb.UpdateExternalCollectionTask, error)); ok {
return rf(ctx)
}
if rf, ok := ret.Get(0).(func(context.Context) []*indexpb.UpdateExternalCollectionTask); ok {
r0 = rf(ctx)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).([]*indexpb.UpdateExternalCollectionTask)
}
}
if rf, ok := ret.Get(1).(func(context.Context) error); ok {
r1 = rf(ctx)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// DataCoordCatalog_ListUpdateExternalCollectionTasks_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ListUpdateExternalCollectionTasks'
type DataCoordCatalog_ListUpdateExternalCollectionTasks_Call struct {
*mock.Call
}
// ListUpdateExternalCollectionTasks is a helper method to define mock.On call
// - ctx context.Context
func (_e *DataCoordCatalog_Expecter) ListUpdateExternalCollectionTasks(ctx interface{}) *DataCoordCatalog_ListUpdateExternalCollectionTasks_Call {
return &DataCoordCatalog_ListUpdateExternalCollectionTasks_Call{Call: _e.mock.On("ListUpdateExternalCollectionTasks", ctx)}
}
func (_c *DataCoordCatalog_ListUpdateExternalCollectionTasks_Call) Run(run func(ctx context.Context)) *DataCoordCatalog_ListUpdateExternalCollectionTasks_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context))
})
return _c
}
func (_c *DataCoordCatalog_ListUpdateExternalCollectionTasks_Call) Return(_a0 []*indexpb.UpdateExternalCollectionTask, _a1 error) *DataCoordCatalog_ListUpdateExternalCollectionTasks_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *DataCoordCatalog_ListUpdateExternalCollectionTasks_Call) RunAndReturn(run func(context.Context) ([]*indexpb.UpdateExternalCollectionTask, error)) *DataCoordCatalog_ListUpdateExternalCollectionTasks_Call {
_c.Call.Return(run)
return _c
}
// MarkChannelAdded provides a mock function with given fields: ctx, channel
func (_m *DataCoordCatalog) MarkChannelAdded(ctx context.Context, channel string) error {
ret := _m.Called(ctx, channel)
@ -2504,6 +2609,53 @@ func (_c *DataCoordCatalog_SaveStatsTask_Call) RunAndReturn(run func(context.Con
return _c
}
// SaveUpdateExternalCollectionTask provides a mock function with given fields: ctx, task
func (_m *DataCoordCatalog) SaveUpdateExternalCollectionTask(ctx context.Context, task *indexpb.UpdateExternalCollectionTask) error {
ret := _m.Called(ctx, task)
if len(ret) == 0 {
panic("no return value specified for SaveUpdateExternalCollectionTask")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.UpdateExternalCollectionTask) error); ok {
r0 = rf(ctx, task)
} else {
r0 = ret.Error(0)
}
return r0
}
// DataCoordCatalog_SaveUpdateExternalCollectionTask_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SaveUpdateExternalCollectionTask'
type DataCoordCatalog_SaveUpdateExternalCollectionTask_Call struct {
*mock.Call
}
// SaveUpdateExternalCollectionTask is a helper method to define mock.On call
// - ctx context.Context
// - task *indexpb.UpdateExternalCollectionTask
func (_e *DataCoordCatalog_Expecter) SaveUpdateExternalCollectionTask(ctx interface{}, task interface{}) *DataCoordCatalog_SaveUpdateExternalCollectionTask_Call {
return &DataCoordCatalog_SaveUpdateExternalCollectionTask_Call{Call: _e.mock.On("SaveUpdateExternalCollectionTask", ctx, task)}
}
func (_c *DataCoordCatalog_SaveUpdateExternalCollectionTask_Call) Run(run func(ctx context.Context, task *indexpb.UpdateExternalCollectionTask)) *DataCoordCatalog_SaveUpdateExternalCollectionTask_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*indexpb.UpdateExternalCollectionTask))
})
return _c
}
func (_c *DataCoordCatalog_SaveUpdateExternalCollectionTask_Call) Return(_a0 error) *DataCoordCatalog_SaveUpdateExternalCollectionTask_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *DataCoordCatalog_SaveUpdateExternalCollectionTask_Call) RunAndReturn(run func(context.Context, *indexpb.UpdateExternalCollectionTask) error) *DataCoordCatalog_SaveUpdateExternalCollectionTask_Call {
_c.Call.Return(run)
return _c
}
// ShouldDropChannel provides a mock function with given fields: ctx, channel
func (_m *DataCoordCatalog) ShouldDropChannel(ctx context.Context, channel string) bool {
ret := _m.Called(ctx, channel)

View File

@ -18,6 +18,8 @@ import (
mock "github.com/stretchr/testify/mock"
msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
txnkv "github.com/tikv/client-go/v2/txnkv"
types "github.com/milvus-io/milvus/internal/types"
@ -390,6 +392,65 @@ func (_c *MockDataCoord_CheckHealth_Call) RunAndReturn(run func(context.Context,
return _c
}
// CreateExternalCollection provides a mock function with given fields: _a0, _a1
func (_m *MockDataCoord) CreateExternalCollection(_a0 context.Context, _a1 *msgpb.CreateCollectionRequest) (*datapb.CreateExternalCollectionResponse, error) {
ret := _m.Called(_a0, _a1)
if len(ret) == 0 {
panic("no return value specified for CreateExternalCollection")
}
var r0 *datapb.CreateExternalCollectionResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *msgpb.CreateCollectionRequest) (*datapb.CreateExternalCollectionResponse, error)); ok {
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *msgpb.CreateCollectionRequest) *datapb.CreateExternalCollectionResponse); ok {
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*datapb.CreateExternalCollectionResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *msgpb.CreateCollectionRequest) error); ok {
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockDataCoord_CreateExternalCollection_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CreateExternalCollection'
type MockDataCoord_CreateExternalCollection_Call struct {
*mock.Call
}
// CreateExternalCollection is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *msgpb.CreateCollectionRequest
func (_e *MockDataCoord_Expecter) CreateExternalCollection(_a0 interface{}, _a1 interface{}) *MockDataCoord_CreateExternalCollection_Call {
return &MockDataCoord_CreateExternalCollection_Call{Call: _e.mock.On("CreateExternalCollection", _a0, _a1)}
}
func (_c *MockDataCoord_CreateExternalCollection_Call) Run(run func(_a0 context.Context, _a1 *msgpb.CreateCollectionRequest)) *MockDataCoord_CreateExternalCollection_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*msgpb.CreateCollectionRequest))
})
return _c
}
func (_c *MockDataCoord_CreateExternalCollection_Call) Return(_a0 *datapb.CreateExternalCollectionResponse, _a1 error) *MockDataCoord_CreateExternalCollection_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockDataCoord_CreateExternalCollection_Call) RunAndReturn(run func(context.Context, *msgpb.CreateCollectionRequest) (*datapb.CreateExternalCollectionResponse, error)) *MockDataCoord_CreateExternalCollection_Call {
_c.Call.Return(run)
return _c
}
// CreateIndex provides a mock function with given fields: _a0, _a1
func (_m *MockDataCoord) CreateIndex(_a0 context.Context, _a1 *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
ret := _m.Called(_a0, _a1)

View File

@ -18,6 +18,8 @@ import (
milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
mock "github.com/stretchr/testify/mock"
msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
)
// MockDataCoordClient is an autogenerated mock type for the DataCoordClient type
@ -522,6 +524,80 @@ func (_c *MockDataCoordClient_Close_Call) RunAndReturn(run func() error) *MockDa
return _c
}
// CreateExternalCollection provides a mock function with given fields: ctx, in, opts
func (_m *MockDataCoordClient) CreateExternalCollection(ctx context.Context, in *msgpb.CreateCollectionRequest, opts ...grpc.CallOption) (*datapb.CreateExternalCollectionResponse, error) {
_va := make([]interface{}, len(opts))
for _i := range opts {
_va[_i] = opts[_i]
}
var _ca []interface{}
_ca = append(_ca, ctx, in)
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
if len(ret) == 0 {
panic("no return value specified for CreateExternalCollection")
}
var r0 *datapb.CreateExternalCollectionResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *msgpb.CreateCollectionRequest, ...grpc.CallOption) (*datapb.CreateExternalCollectionResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *msgpb.CreateCollectionRequest, ...grpc.CallOption) *datapb.CreateExternalCollectionResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*datapb.CreateExternalCollectionResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *msgpb.CreateCollectionRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockDataCoordClient_CreateExternalCollection_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CreateExternalCollection'
type MockDataCoordClient_CreateExternalCollection_Call struct {
*mock.Call
}
// CreateExternalCollection is a helper method to define mock.On call
// - ctx context.Context
// - in *msgpb.CreateCollectionRequest
// - opts ...grpc.CallOption
func (_e *MockDataCoordClient_Expecter) CreateExternalCollection(ctx interface{}, in interface{}, opts ...interface{}) *MockDataCoordClient_CreateExternalCollection_Call {
return &MockDataCoordClient_CreateExternalCollection_Call{Call: _e.mock.On("CreateExternalCollection",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockDataCoordClient_CreateExternalCollection_Call) Run(run func(ctx context.Context, in *msgpb.CreateCollectionRequest, opts ...grpc.CallOption)) *MockDataCoordClient_CreateExternalCollection_Call {
_c.Call.Run(func(args mock.Arguments) {
variadicArgs := make([]grpc.CallOption, len(args)-2)
for i, a := range args[2:] {
if a != nil {
variadicArgs[i] = a.(grpc.CallOption)
}
}
run(args[0].(context.Context), args[1].(*msgpb.CreateCollectionRequest), variadicArgs...)
})
return _c
}
func (_c *MockDataCoordClient_CreateExternalCollection_Call) Return(_a0 *datapb.CreateExternalCollectionResponse, _a1 error) *MockDataCoordClient_CreateExternalCollection_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockDataCoordClient_CreateExternalCollection_Call) RunAndReturn(run func(context.Context, *msgpb.CreateCollectionRequest, ...grpc.CallOption) (*datapb.CreateExternalCollectionResponse, error)) *MockDataCoordClient_CreateExternalCollection_Call {
_c.Call.Return(run)
return _c
}
// CreateIndex provides a mock function with given fields: ctx, in, opts
func (_m *MockDataCoordClient) CreateIndex(ctx context.Context, in *indexpb.CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
_va := make([]interface{}, len(opts))

View File

@ -20,6 +20,8 @@ import (
mock "github.com/stretchr/testify/mock"
msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
proxypb "github.com/milvus-io/milvus/pkg/v2/proto/proxypb"
querypb "github.com/milvus-io/milvus/pkg/v2/proto/querypb"
@ -1401,6 +1403,65 @@ func (_c *MixCoord_CreateDatabase_Call) RunAndReturn(run func(context.Context, *
return _c
}
// CreateExternalCollection provides a mock function with given fields: _a0, _a1
func (_m *MixCoord) CreateExternalCollection(_a0 context.Context, _a1 *msgpb.CreateCollectionRequest) (*datapb.CreateExternalCollectionResponse, error) {
ret := _m.Called(_a0, _a1)
if len(ret) == 0 {
panic("no return value specified for CreateExternalCollection")
}
var r0 *datapb.CreateExternalCollectionResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *msgpb.CreateCollectionRequest) (*datapb.CreateExternalCollectionResponse, error)); ok {
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *msgpb.CreateCollectionRequest) *datapb.CreateExternalCollectionResponse); ok {
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*datapb.CreateExternalCollectionResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *msgpb.CreateCollectionRequest) error); ok {
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MixCoord_CreateExternalCollection_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CreateExternalCollection'
type MixCoord_CreateExternalCollection_Call struct {
*mock.Call
}
// CreateExternalCollection is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *msgpb.CreateCollectionRequest
func (_e *MixCoord_Expecter) CreateExternalCollection(_a0 interface{}, _a1 interface{}) *MixCoord_CreateExternalCollection_Call {
return &MixCoord_CreateExternalCollection_Call{Call: _e.mock.On("CreateExternalCollection", _a0, _a1)}
}
func (_c *MixCoord_CreateExternalCollection_Call) Run(run func(_a0 context.Context, _a1 *msgpb.CreateCollectionRequest)) *MixCoord_CreateExternalCollection_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*msgpb.CreateCollectionRequest))
})
return _c
}
func (_c *MixCoord_CreateExternalCollection_Call) Return(_a0 *datapb.CreateExternalCollectionResponse, _a1 error) *MixCoord_CreateExternalCollection_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MixCoord_CreateExternalCollection_Call) RunAndReturn(run func(context.Context, *msgpb.CreateCollectionRequest) (*datapb.CreateExternalCollectionResponse, error)) *MixCoord_CreateExternalCollection_Call {
_c.Call.Return(run)
return _c
}
// CreateIndex provides a mock function with given fields: _a0, _a1
func (_m *MixCoord) CreateIndex(_a0 context.Context, _a1 *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
ret := _m.Called(_a0, _a1)

View File

@ -19,6 +19,8 @@ import (
mock "github.com/stretchr/testify/mock"
msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
proxypb "github.com/milvus-io/milvus/pkg/v2/proto/proxypb"
querypb "github.com/milvus-io/milvus/pkg/v2/proto/querypb"
@ -1786,6 +1788,80 @@ func (_c *MockMixCoordClient_CreateDatabase_Call) RunAndReturn(run func(context.
return _c
}
// CreateExternalCollection provides a mock function with given fields: ctx, in, opts
func (_m *MockMixCoordClient) CreateExternalCollection(ctx context.Context, in *msgpb.CreateCollectionRequest, opts ...grpc.CallOption) (*datapb.CreateExternalCollectionResponse, error) {
_va := make([]interface{}, len(opts))
for _i := range opts {
_va[_i] = opts[_i]
}
var _ca []interface{}
_ca = append(_ca, ctx, in)
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
if len(ret) == 0 {
panic("no return value specified for CreateExternalCollection")
}
var r0 *datapb.CreateExternalCollectionResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *msgpb.CreateCollectionRequest, ...grpc.CallOption) (*datapb.CreateExternalCollectionResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *msgpb.CreateCollectionRequest, ...grpc.CallOption) *datapb.CreateExternalCollectionResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*datapb.CreateExternalCollectionResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *msgpb.CreateCollectionRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockMixCoordClient_CreateExternalCollection_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CreateExternalCollection'
type MockMixCoordClient_CreateExternalCollection_Call struct {
*mock.Call
}
// CreateExternalCollection is a helper method to define mock.On call
// - ctx context.Context
// - in *msgpb.CreateCollectionRequest
// - opts ...grpc.CallOption
func (_e *MockMixCoordClient_Expecter) CreateExternalCollection(ctx interface{}, in interface{}, opts ...interface{}) *MockMixCoordClient_CreateExternalCollection_Call {
return &MockMixCoordClient_CreateExternalCollection_Call{Call: _e.mock.On("CreateExternalCollection",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockMixCoordClient_CreateExternalCollection_Call) Run(run func(ctx context.Context, in *msgpb.CreateCollectionRequest, opts ...grpc.CallOption)) *MockMixCoordClient_CreateExternalCollection_Call {
_c.Call.Run(func(args mock.Arguments) {
variadicArgs := make([]grpc.CallOption, len(args)-2)
for i, a := range args[2:] {
if a != nil {
variadicArgs[i] = a.(grpc.CallOption)
}
}
run(args[0].(context.Context), args[1].(*msgpb.CreateCollectionRequest), variadicArgs...)
})
return _c
}
func (_c *MockMixCoordClient_CreateExternalCollection_Call) Return(_a0 *datapb.CreateExternalCollectionResponse, _a1 error) *MockMixCoordClient_CreateExternalCollection_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockMixCoordClient_CreateExternalCollection_Call) RunAndReturn(run func(context.Context, *msgpb.CreateCollectionRequest, ...grpc.CallOption) (*datapb.CreateExternalCollectionResponse, error)) *MockMixCoordClient_CreateExternalCollection_Call {
_c.Call.Return(run)
return _c
}
// CreateIndex provides a mock function with given fields: ctx, in, opts
func (_m *MockMixCoordClient) CreateIndex(ctx context.Context, in *indexpb.CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
_va := make([]interface{}, len(opts))

View File

@ -476,6 +476,52 @@ func (_c *MockCSegment_Insert_Call) RunAndReturn(run func(context.Context, *segc
return _c
}
// Load provides a mock function with given fields: ctx
func (_m *MockCSegment) Load(ctx context.Context) error {
ret := _m.Called(ctx)
if len(ret) == 0 {
panic("no return value specified for Load")
}
var r0 error
if rf, ok := ret.Get(0).(func(context.Context) error); ok {
r0 = rf(ctx)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockCSegment_Load_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Load'
type MockCSegment_Load_Call struct {
*mock.Call
}
// Load is a helper method to define mock.On call
// - ctx context.Context
func (_e *MockCSegment_Expecter) Load(ctx interface{}) *MockCSegment_Load_Call {
return &MockCSegment_Load_Call{Call: _e.mock.On("Load", ctx)}
}
func (_c *MockCSegment_Load_Call) Run(run func(ctx context.Context)) *MockCSegment_Load_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context))
})
return _c
}
func (_c *MockCSegment_Load_Call) Return(_a0 error) *MockCSegment_Load_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockCSegment_Load_Call) RunAndReturn(run func(context.Context) error) *MockCSegment_Load_Call {
_c.Call.Return(run)
return _c
}
// LoadFieldData provides a mock function with given fields: ctx, request
func (_m *MockCSegment) LoadFieldData(ctx context.Context, request *segcore.LoadFieldDataRequest) (*segcore.LoadFieldDataResult, error) {
ret := _m.Called(ctx, request)

View File

@ -29,6 +29,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/msgpb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
@ -1664,6 +1665,12 @@ func (coord *MixCoordMock) ValidateAnalyzer(ctx context.Context, req *querypb.Va
return merr.Success(), nil
}
func (coord *MixCoordMock) CreateExternalCollection(ctx context.Context, req *msgpb.CreateCollectionRequest, opts ...grpc.CallOption) (*datapb.CreateExternalCollectionResponse, error) {
return &datapb.CreateExternalCollectionResponse{
Status: merr.Success(),
}, nil
}
type DescribeCollectionFunc func(ctx context.Context, request *milvuspb.DescribeCollectionRequest, opts ...grpc.CallOption) (*milvuspb.DescribeCollectionResponse, error)
type ShowPartitionsFunc func(ctx context.Context, request *milvuspb.ShowPartitionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionsResponse, error)

View File

@ -32,6 +32,8 @@ service DataCoord {
rpc Flush(FlushRequest) returns (FlushResponse) {}
rpc FlushAll(FlushAllRequest) returns(FlushAllResponse) {}
rpc CreateExternalCollection(msg.CreateCollectionRequest) returns (CreateExternalCollectionResponse) {}
// AllocSegment alloc a new growing segment, add it into segment meta.
rpc AllocSegment(AllocSegmentRequest) returns (AllocSegmentResponse) {}
@ -1137,3 +1139,7 @@ message FileResourceInfo {
string path = 2;
int64 resource_id = 3;
}
message CreateExternalCollectionResponse {
common.Status status = 1;
}

File diff suppressed because it is too large Load Diff

View File

@ -10,6 +10,7 @@ import (
context "context"
commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
indexpb "github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
internalpb "github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
grpc "google.golang.org/grpc"
@ -25,6 +26,7 @@ const _ = grpc.SupportPackageIsVersion7
const (
DataCoord_Flush_FullMethodName = "/milvus.proto.data.DataCoord/Flush"
DataCoord_FlushAll_FullMethodName = "/milvus.proto.data.DataCoord/FlushAll"
DataCoord_CreateExternalCollection_FullMethodName = "/milvus.proto.data.DataCoord/CreateExternalCollection"
DataCoord_AllocSegment_FullMethodName = "/milvus.proto.data.DataCoord/AllocSegment"
DataCoord_AssignSegmentID_FullMethodName = "/milvus.proto.data.DataCoord/AssignSegmentID"
DataCoord_GetSegmentInfo_FullMethodName = "/milvus.proto.data.DataCoord/GetSegmentInfo"
@ -81,6 +83,7 @@ const (
type DataCoordClient interface {
Flush(ctx context.Context, in *FlushRequest, opts ...grpc.CallOption) (*FlushResponse, error)
FlushAll(ctx context.Context, in *FlushAllRequest, opts ...grpc.CallOption) (*FlushAllResponse, error)
CreateExternalCollection(ctx context.Context, in *msgpb.CreateCollectionRequest, opts ...grpc.CallOption) (*CreateExternalCollectionResponse, error)
// AllocSegment alloc a new growing segment, add it into segment meta.
AllocSegment(ctx context.Context, in *AllocSegmentRequest, opts ...grpc.CallOption) (*AllocSegmentResponse, error)
// Deprecated: Do not use.
@ -166,6 +169,15 @@ func (c *dataCoordClient) FlushAll(ctx context.Context, in *FlushAllRequest, opt
return out, nil
}
func (c *dataCoordClient) CreateExternalCollection(ctx context.Context, in *msgpb.CreateCollectionRequest, opts ...grpc.CallOption) (*CreateExternalCollectionResponse, error) {
out := new(CreateExternalCollectionResponse)
err := c.cc.Invoke(ctx, DataCoord_CreateExternalCollection_FullMethodName, in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *dataCoordClient) AllocSegment(ctx context.Context, in *AllocSegmentRequest, opts ...grpc.CallOption) (*AllocSegmentResponse, error) {
out := new(AllocSegmentResponse)
err := c.cc.Invoke(ctx, DataCoord_AllocSegment_FullMethodName, in, out, opts...)
@ -606,6 +618,7 @@ func (c *dataCoordClient) ListFileResources(ctx context.Context, in *milvuspb.Li
type DataCoordServer interface {
Flush(context.Context, *FlushRequest) (*FlushResponse, error)
FlushAll(context.Context, *FlushAllRequest) (*FlushAllResponse, error)
CreateExternalCollection(context.Context, *msgpb.CreateCollectionRequest) (*CreateExternalCollectionResponse, error)
// AllocSegment alloc a new growing segment, add it into segment meta.
AllocSegment(context.Context, *AllocSegmentRequest) (*AllocSegmentResponse, error)
// Deprecated: Do not use.
@ -675,6 +688,9 @@ func (UnimplementedDataCoordServer) Flush(context.Context, *FlushRequest) (*Flus
func (UnimplementedDataCoordServer) FlushAll(context.Context, *FlushAllRequest) (*FlushAllResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method FlushAll not implemented")
}
func (UnimplementedDataCoordServer) CreateExternalCollection(context.Context, *msgpb.CreateCollectionRequest) (*CreateExternalCollectionResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method CreateExternalCollection not implemented")
}
func (UnimplementedDataCoordServer) AllocSegment(context.Context, *AllocSegmentRequest) (*AllocSegmentResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method AllocSegment not implemented")
}
@ -867,6 +883,24 @@ func _DataCoord_FlushAll_Handler(srv interface{}, ctx context.Context, dec func(
return interceptor(ctx, in, info, handler)
}
func _DataCoord_CreateExternalCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(msgpb.CreateCollectionRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(DataCoordServer).CreateExternalCollection(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: DataCoord_CreateExternalCollection_FullMethodName,
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(DataCoordServer).CreateExternalCollection(ctx, req.(*msgpb.CreateCollectionRequest))
}
return interceptor(ctx, in, info, handler)
}
func _DataCoord_AllocSegment_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(AllocSegmentRequest)
if err := dec(in); err != nil {
@ -1746,6 +1780,10 @@ var DataCoord_ServiceDesc = grpc.ServiceDesc{
MethodName: "FlushAll",
Handler: _DataCoord_FlushAll_Handler,
},
{
MethodName: "CreateExternalCollection",
Handler: _DataCoord_CreateExternalCollection_Handler,
},
{
MethodName: "AllocSegment",
Handler: _DataCoord_AllocSegment_Handler,

View File

@ -325,6 +325,17 @@ message StatsTask {
bool canRecycle = 12;
}
message UpdateExternalCollectionTask {
int64 collectionID = 1;
int64 taskID = 2;
int64 version = 3;
int64 nodeID = 4;
JobState state = 5;
string fail_reason = 6;
string external_source = 7;
string external_spec = 8;
}
enum StatsSubJob {
None = 0;
// deprecated, after v2.6.0, move to compaction

View File

@ -2827,6 +2827,109 @@ func (x *StatsTask) GetCanRecycle() bool {
return false
}
type UpdateExternalCollectionTask struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"`
TaskID int64 `protobuf:"varint,2,opt,name=taskID,proto3" json:"taskID,omitempty"`
Version int64 `protobuf:"varint,3,opt,name=version,proto3" json:"version,omitempty"`
NodeID int64 `protobuf:"varint,4,opt,name=nodeID,proto3" json:"nodeID,omitempty"`
State JobState `protobuf:"varint,5,opt,name=state,proto3,enum=milvus.proto.index.JobState" json:"state,omitempty"`
FailReason string `protobuf:"bytes,6,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"`
ExternalSource string `protobuf:"bytes,7,opt,name=external_source,json=externalSource,proto3" json:"external_source,omitempty"`
ExternalSpec string `protobuf:"bytes,8,opt,name=external_spec,json=externalSpec,proto3" json:"external_spec,omitempty"`
}
func (x *UpdateExternalCollectionTask) Reset() {
*x = UpdateExternalCollectionTask{}
if protoimpl.UnsafeEnabled {
mi := &file_index_coord_proto_msgTypes[32]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *UpdateExternalCollectionTask) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*UpdateExternalCollectionTask) ProtoMessage() {}
func (x *UpdateExternalCollectionTask) ProtoReflect() protoreflect.Message {
mi := &file_index_coord_proto_msgTypes[32]
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 UpdateExternalCollectionTask.ProtoReflect.Descriptor instead.
func (*UpdateExternalCollectionTask) Descriptor() ([]byte, []int) {
return file_index_coord_proto_rawDescGZIP(), []int{32}
}
func (x *UpdateExternalCollectionTask) GetCollectionID() int64 {
if x != nil {
return x.CollectionID
}
return 0
}
func (x *UpdateExternalCollectionTask) GetTaskID() int64 {
if x != nil {
return x.TaskID
}
return 0
}
func (x *UpdateExternalCollectionTask) GetVersion() int64 {
if x != nil {
return x.Version
}
return 0
}
func (x *UpdateExternalCollectionTask) GetNodeID() int64 {
if x != nil {
return x.NodeID
}
return 0
}
func (x *UpdateExternalCollectionTask) GetState() JobState {
if x != nil {
return x.State
}
return JobState_JobStateNone
}
func (x *UpdateExternalCollectionTask) GetFailReason() string {
if x != nil {
return x.FailReason
}
return ""
}
func (x *UpdateExternalCollectionTask) GetExternalSource() string {
if x != nil {
return x.ExternalSource
}
return ""
}
func (x *UpdateExternalCollectionTask) GetExternalSpec() string {
if x != nil {
return x.ExternalSpec
}
return ""
}
var File_index_coord_proto protoreflect.FileDescriptor
var file_index_coord_proto_rawDesc = []byte{
@ -3308,109 +3411,128 @@ var file_index_coord_proto_rawDesc = []byte{
0x75, 0x62, 0x4a, 0x6f, 0x62, 0x52, 0x0a, 0x73, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70,
0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x61, 0x6e, 0x52, 0x65, 0x63, 0x79, 0x63, 0x6c, 0x65, 0x18,
0x0c, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x63, 0x61, 0x6e, 0x52, 0x65, 0x63, 0x79, 0x63, 0x6c,
0x65, 0x2a, 0x5b, 0x0a, 0x07, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b,
0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x13, 0x0a,
0x0f, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62,
0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x41, 0x6e, 0x61,
0x6c, 0x79, 0x7a, 0x65, 0x4a, 0x6f, 0x62, 0x10, 0x02, 0x12, 0x13, 0x0a, 0x0f, 0x4a, 0x6f, 0x62,
0x54, 0x79, 0x70, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4a, 0x6f, 0x62, 0x10, 0x03, 0x2a, 0x83,
0x01, 0x0a, 0x08, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x10, 0x0a, 0x0c, 0x4a,
0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x10, 0x0a,
0x0c, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x69, 0x74, 0x10, 0x01, 0x12,
0x16, 0x0a, 0x12, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x50, 0x72, 0x6f,
0x67, 0x72, 0x65, 0x73, 0x73, 0x10, 0x02, 0x12, 0x14, 0x0a, 0x10, 0x4a, 0x6f, 0x62, 0x53, 0x74,
0x61, 0x74, 0x65, 0x46, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x10, 0x03, 0x12, 0x12, 0x0a,
0x0e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10,
0x04, 0x12, 0x11, 0x0a, 0x0d, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x74,
0x72, 0x79, 0x10, 0x05, 0x2a, 0x55, 0x0a, 0x0b, 0x53, 0x74, 0x61, 0x74, 0x73, 0x53, 0x75, 0x62,
0x4a, 0x6f, 0x62, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x08, 0x0a,
0x04, 0x53, 0x6f, 0x72, 0x74, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x65, 0x78, 0x74, 0x49,
0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x42, 0x4d, 0x32,
0x35, 0x4a, 0x6f, 0x62, 0x10, 0x03, 0x12, 0x13, 0x0a, 0x0f, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65,
0x79, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x10, 0x04, 0x32, 0xf4, 0x09, 0x0a, 0x0a,
0x49, 0x6e, 0x64, 0x65, 0x78, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x12, 0x54, 0x0a, 0x0b, 0x43, 0x72,
0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43,
0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x1a, 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, 0x22, 0x00,
0x12, 0x52, 0x0a, 0x0a, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x25,
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e,
0x64, 0x65, 0x78, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78,
0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
0x65, 0x22, 0xaf, 0x02, 0x0a, 0x1c, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x78, 0x74, 0x65,
0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61,
0x73, 0x6b, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44,
0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x18,
0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52,
0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65,
0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44,
0x12, 0x32, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32,
0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69,
0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73,
0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61,
0x73, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52,
0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x27, 0x0a, 0x0f, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61,
0x6c, 0x5f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e,
0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x23,
0x0a, 0x0d, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18,
0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x53,
0x70, 0x65, 0x63, 0x2a, 0x5b, 0x0a, 0x07, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f,
0x0a, 0x0b, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12,
0x13, 0x0a, 0x0f, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a,
0x6f, 0x62, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x41,
0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x4a, 0x6f, 0x62, 0x10, 0x02, 0x12, 0x13, 0x0a, 0x0f, 0x4a,
0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4a, 0x6f, 0x62, 0x10, 0x03,
0x2a, 0x83, 0x01, 0x0a, 0x08, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x10, 0x0a,
0x0c, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12,
0x10, 0x0a, 0x0c, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x69, 0x74, 0x10,
0x01, 0x12, 0x16, 0x0a, 0x12, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x50,
0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x10, 0x02, 0x12, 0x14, 0x0a, 0x10, 0x4a, 0x6f, 0x62,
0x53, 0x74, 0x61, 0x74, 0x65, 0x46, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x10, 0x03, 0x12,
0x12, 0x0a, 0x0e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x46, 0x61, 0x69, 0x6c, 0x65,
0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52,
0x65, 0x74, 0x72, 0x79, 0x10, 0x05, 0x2a, 0x55, 0x0a, 0x0b, 0x53, 0x74, 0x61, 0x74, 0x73, 0x53,
0x75, 0x62, 0x4a, 0x6f, 0x62, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12,
0x08, 0x0a, 0x04, 0x53, 0x6f, 0x72, 0x74, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x65, 0x78,
0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x42,
0x4d, 0x32, 0x35, 0x4a, 0x6f, 0x62, 0x10, 0x03, 0x12, 0x13, 0x0a, 0x0f, 0x4a, 0x73, 0x6f, 0x6e,
0x4b, 0x65, 0x79, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x10, 0x04, 0x32, 0xf4, 0x09,
0x0a, 0x0a, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x12, 0x54, 0x0a, 0x0b,
0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x26, 0x2e, 0x6d, 0x69,
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78,
0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x1a, 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,
0x22, 0x00, 0x12, 0x52, 0x0a, 0x0a, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78,
0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64,
0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74,
0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53,
0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7b,
0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65,
0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x53,
0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74,
0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74,
0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0d, 0x47,
0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x27, 0x2e, 0x6d,
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65,
0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e,
0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69,
0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61,
0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x14,
0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53,
0x74, 0x61, 0x74, 0x65, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67,
0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65,
0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52,
0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0d, 0x47, 0x65, 0x74,
0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c,
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e,
0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f,
0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65,
0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
0x50, 0x0a, 0x09, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x24, 0x2e, 0x6d,
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65,
0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x1a, 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, 0x22,
0x00, 0x12, 0x66, 0x0a, 0x0d, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64,
0x65, 0x78, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65,
0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d,
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65,
0x78, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52,
0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x75, 0x0a, 0x12, 0x47, 0x65, 0x74,
0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12,
0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69,
0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61,
0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e,
0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
0x00, 0x12, 0x50, 0x0a, 0x09, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x24,
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e,
0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74,
0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
0x12, 0x7e, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c,
0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47,
0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67,
0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69,
0x64, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49,
0x6e, 0x64, 0x65, 0x78, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69,
0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29,
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e,
0x64, 0x65, 0x78, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65,
0x78, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x75, 0x0a, 0x12, 0x47,
0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63,
0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53,
0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74,
0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
0x22, 0x00, 0x12, 0x7e, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75,
0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69,
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78,
0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72,
0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
0x12, 0x7b, 0x0a, 0x12, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72,
0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53,
0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e,
0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e,
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64,
0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64,
0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
0x22, 0x00, 0x12, 0x7b, 0x0a, 0x12, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67,
0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c,
0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69,
0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a,
0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69,
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f,
0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74,
0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x62,
0x0a, 0x0b, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12, 0x27, 0x2e,
0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c,
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e,
0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61,
0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x26, 0x2e,
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c,
0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65,
0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
0x22, 0x00, 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, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x69,
0x6e, 0x64, 0x65, 0x78, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d,
0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
0x12, 0x62, 0x0a, 0x0b, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12,
0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d,
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74,
0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43,
0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
0x73, 0x65, 0x22, 0x00, 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, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x2f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@ -3426,7 +3548,7 @@ func file_index_coord_proto_rawDescGZIP() []byte {
}
var file_index_coord_proto_enumTypes = make([]protoimpl.EnumInfo, 3)
var file_index_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 33)
var file_index_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 34)
var file_index_coord_proto_goTypes = []interface{}{
(JobType)(0), // 0: milvus.proto.index.JobType
(JobState)(0), // 1: milvus.proto.index.JobState
@ -3463,87 +3585,89 @@ var file_index_coord_proto_goTypes = []interface{}{
(*SegmentStats)(nil), // 32: milvus.proto.index.SegmentStats
(*FieldLogPath)(nil), // 33: milvus.proto.index.FieldLogPath
(*StatsTask)(nil), // 34: milvus.proto.index.StatsTask
nil, // 35: milvus.proto.index.GetIndexInfoResponse.SegmentInfoEntry
(*commonpb.KeyValuePair)(nil), // 36: milvus.proto.common.KeyValuePair
(commonpb.IndexState)(0), // 37: milvus.proto.common.IndexState
(*commonpb.MsgBase)(nil), // 38: milvus.proto.common.MsgBase
(*commonpb.Address)(nil), // 39: milvus.proto.common.Address
(*commonpb.Status)(nil), // 40: milvus.proto.common.Status
(*internalpb.InitParams)(nil), // 41: milvus.proto.internal.InitParams
(schemapb.DataType)(0), // 42: milvus.proto.schema.DataType
(*internalpb.ShowConfigurationsRequest)(nil), // 43: milvus.proto.internal.ShowConfigurationsRequest
(*milvuspb.GetMetricsRequest)(nil), // 44: milvus.proto.milvus.GetMetricsRequest
(*milvuspb.CheckHealthRequest)(nil), // 45: milvus.proto.milvus.CheckHealthRequest
(*internalpb.ShowConfigurationsResponse)(nil), // 46: milvus.proto.internal.ShowConfigurationsResponse
(*milvuspb.GetMetricsResponse)(nil), // 47: milvus.proto.milvus.GetMetricsResponse
(*milvuspb.CheckHealthResponse)(nil), // 48: milvus.proto.milvus.CheckHealthResponse
(*UpdateExternalCollectionTask)(nil), // 35: milvus.proto.index.UpdateExternalCollectionTask
nil, // 36: milvus.proto.index.GetIndexInfoResponse.SegmentInfoEntry
(*commonpb.KeyValuePair)(nil), // 37: milvus.proto.common.KeyValuePair
(commonpb.IndexState)(0), // 38: milvus.proto.common.IndexState
(*commonpb.MsgBase)(nil), // 39: milvus.proto.common.MsgBase
(*commonpb.Address)(nil), // 40: milvus.proto.common.Address
(*commonpb.Status)(nil), // 41: milvus.proto.common.Status
(*internalpb.InitParams)(nil), // 42: milvus.proto.internal.InitParams
(schemapb.DataType)(0), // 43: milvus.proto.schema.DataType
(*internalpb.ShowConfigurationsRequest)(nil), // 44: milvus.proto.internal.ShowConfigurationsRequest
(*milvuspb.GetMetricsRequest)(nil), // 45: milvus.proto.milvus.GetMetricsRequest
(*milvuspb.CheckHealthRequest)(nil), // 46: milvus.proto.milvus.CheckHealthRequest
(*internalpb.ShowConfigurationsResponse)(nil), // 47: milvus.proto.internal.ShowConfigurationsResponse
(*milvuspb.GetMetricsResponse)(nil), // 48: milvus.proto.milvus.GetMetricsResponse
(*milvuspb.CheckHealthResponse)(nil), // 49: milvus.proto.milvus.CheckHealthResponse
}
var file_index_coord_proto_depIdxs = []int32{
36, // 0: milvus.proto.index.IndexInfo.type_params:type_name -> milvus.proto.common.KeyValuePair
36, // 1: milvus.proto.index.IndexInfo.index_params:type_name -> milvus.proto.common.KeyValuePair
37, // 2: milvus.proto.index.IndexInfo.state:type_name -> milvus.proto.common.IndexState
36, // 3: milvus.proto.index.IndexInfo.user_index_params:type_name -> milvus.proto.common.KeyValuePair
37, // 0: milvus.proto.index.IndexInfo.type_params:type_name -> milvus.proto.common.KeyValuePair
37, // 1: milvus.proto.index.IndexInfo.index_params:type_name -> milvus.proto.common.KeyValuePair
38, // 2: milvus.proto.index.IndexInfo.state:type_name -> milvus.proto.common.IndexState
37, // 3: milvus.proto.index.IndexInfo.user_index_params:type_name -> milvus.proto.common.KeyValuePair
3, // 4: milvus.proto.index.FieldIndex.index_info:type_name -> milvus.proto.index.IndexInfo
37, // 5: milvus.proto.index.SegmentIndex.state:type_name -> milvus.proto.common.IndexState
38, // 6: milvus.proto.index.RegisterNodeRequest.base:type_name -> milvus.proto.common.MsgBase
39, // 7: milvus.proto.index.RegisterNodeRequest.address:type_name -> milvus.proto.common.Address
40, // 8: milvus.proto.index.RegisterNodeResponse.status:type_name -> milvus.proto.common.Status
41, // 9: milvus.proto.index.RegisterNodeResponse.init_params:type_name -> milvus.proto.internal.InitParams
40, // 10: milvus.proto.index.GetIndexStateResponse.status:type_name -> milvus.proto.common.Status
37, // 11: milvus.proto.index.GetIndexStateResponse.state:type_name -> milvus.proto.common.IndexState
37, // 12: milvus.proto.index.SegmentIndexState.state:type_name -> milvus.proto.common.IndexState
40, // 13: milvus.proto.index.GetSegmentIndexStateResponse.status:type_name -> milvus.proto.common.Status
38, // 5: milvus.proto.index.SegmentIndex.state:type_name -> milvus.proto.common.IndexState
39, // 6: milvus.proto.index.RegisterNodeRequest.base:type_name -> milvus.proto.common.MsgBase
40, // 7: milvus.proto.index.RegisterNodeRequest.address:type_name -> milvus.proto.common.Address
41, // 8: milvus.proto.index.RegisterNodeResponse.status:type_name -> milvus.proto.common.Status
42, // 9: milvus.proto.index.RegisterNodeResponse.init_params:type_name -> milvus.proto.internal.InitParams
41, // 10: milvus.proto.index.GetIndexStateResponse.status:type_name -> milvus.proto.common.Status
38, // 11: milvus.proto.index.GetIndexStateResponse.state:type_name -> milvus.proto.common.IndexState
38, // 12: milvus.proto.index.SegmentIndexState.state:type_name -> milvus.proto.common.IndexState
41, // 13: milvus.proto.index.GetSegmentIndexStateResponse.status:type_name -> milvus.proto.common.Status
11, // 14: milvus.proto.index.GetSegmentIndexStateResponse.states:type_name -> milvus.proto.index.SegmentIndexState
36, // 15: milvus.proto.index.CreateIndexRequest.type_params:type_name -> milvus.proto.common.KeyValuePair
36, // 16: milvus.proto.index.CreateIndexRequest.index_params:type_name -> milvus.proto.common.KeyValuePair
36, // 17: milvus.proto.index.CreateIndexRequest.user_index_params:type_name -> milvus.proto.common.KeyValuePair
36, // 18: milvus.proto.index.AlterIndexRequest.params:type_name -> milvus.proto.common.KeyValuePair
36, // 19: milvus.proto.index.IndexFilePathInfo.index_params:type_name -> milvus.proto.common.KeyValuePair
37, // 15: milvus.proto.index.CreateIndexRequest.type_params:type_name -> milvus.proto.common.KeyValuePair
37, // 16: milvus.proto.index.CreateIndexRequest.index_params:type_name -> milvus.proto.common.KeyValuePair
37, // 17: milvus.proto.index.CreateIndexRequest.user_index_params:type_name -> milvus.proto.common.KeyValuePair
37, // 18: milvus.proto.index.AlterIndexRequest.params:type_name -> milvus.proto.common.KeyValuePair
37, // 19: milvus.proto.index.IndexFilePathInfo.index_params:type_name -> milvus.proto.common.KeyValuePair
16, // 20: milvus.proto.index.SegmentInfo.index_infos:type_name -> milvus.proto.index.IndexFilePathInfo
40, // 21: milvus.proto.index.GetIndexInfoResponse.status:type_name -> milvus.proto.common.Status
35, // 22: milvus.proto.index.GetIndexInfoResponse.segment_info:type_name -> milvus.proto.index.GetIndexInfoResponse.SegmentInfoEntry
40, // 23: milvus.proto.index.DescribeIndexResponse.status:type_name -> milvus.proto.common.Status
41, // 21: milvus.proto.index.GetIndexInfoResponse.status:type_name -> milvus.proto.common.Status
36, // 22: milvus.proto.index.GetIndexInfoResponse.segment_info:type_name -> milvus.proto.index.GetIndexInfoResponse.SegmentInfoEntry
41, // 23: milvus.proto.index.DescribeIndexResponse.status:type_name -> milvus.proto.common.Status
3, // 24: milvus.proto.index.DescribeIndexResponse.index_infos:type_name -> milvus.proto.index.IndexInfo
40, // 25: milvus.proto.index.GetIndexBuildProgressResponse.status:type_name -> milvus.proto.common.Status
36, // 26: milvus.proto.index.JobInfo.index_params:type_name -> milvus.proto.common.KeyValuePair
40, // 27: milvus.proto.index.GetIndexStatisticsResponse.status:type_name -> milvus.proto.common.Status
41, // 25: milvus.proto.index.GetIndexBuildProgressResponse.status:type_name -> milvus.proto.common.Status
37, // 26: milvus.proto.index.JobInfo.index_params:type_name -> milvus.proto.common.KeyValuePair
41, // 27: milvus.proto.index.GetIndexStatisticsResponse.status:type_name -> milvus.proto.common.Status
3, // 28: milvus.proto.index.GetIndexStatisticsResponse.index_infos:type_name -> milvus.proto.index.IndexInfo
40, // 29: milvus.proto.index.ListIndexesResponse.status:type_name -> milvus.proto.common.Status
41, // 29: milvus.proto.index.ListIndexesResponse.status:type_name -> milvus.proto.common.Status
3, // 30: milvus.proto.index.ListIndexesResponse.index_infos:type_name -> milvus.proto.index.IndexInfo
42, // 31: milvus.proto.index.AnalyzeTask.field_type:type_name -> milvus.proto.schema.DataType
43, // 31: milvus.proto.index.AnalyzeTask.field_type:type_name -> milvus.proto.schema.DataType
1, // 32: milvus.proto.index.AnalyzeTask.state:type_name -> milvus.proto.index.JobState
1, // 33: milvus.proto.index.StatsTask.state:type_name -> milvus.proto.index.JobState
2, // 34: milvus.proto.index.StatsTask.subJobType:type_name -> milvus.proto.index.StatsSubJob
17, // 35: milvus.proto.index.GetIndexInfoResponse.SegmentInfoEntry.value:type_name -> milvus.proto.index.SegmentInfo
13, // 36: milvus.proto.index.IndexCoord.CreateIndex:input_type -> milvus.proto.index.CreateIndexRequest
14, // 37: milvus.proto.index.IndexCoord.AlterIndex:input_type -> milvus.proto.index.AlterIndexRequest
8, // 38: milvus.proto.index.IndexCoord.GetIndexState:input_type -> milvus.proto.index.GetIndexStateRequest
10, // 39: milvus.proto.index.IndexCoord.GetSegmentIndexState:input_type -> milvus.proto.index.GetSegmentIndexStateRequest
15, // 40: milvus.proto.index.IndexCoord.GetIndexInfos:input_type -> milvus.proto.index.GetIndexInfoRequest
19, // 41: milvus.proto.index.IndexCoord.DropIndex:input_type -> milvus.proto.index.DropIndexRequest
20, // 42: milvus.proto.index.IndexCoord.DescribeIndex:input_type -> milvus.proto.index.DescribeIndexRequest
27, // 43: milvus.proto.index.IndexCoord.GetIndexStatistics:input_type -> milvus.proto.index.GetIndexStatisticsRequest
22, // 44: milvus.proto.index.IndexCoord.GetIndexBuildProgress:input_type -> milvus.proto.index.GetIndexBuildProgressRequest
43, // 45: milvus.proto.index.IndexCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest
44, // 46: milvus.proto.index.IndexCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest
45, // 47: milvus.proto.index.IndexCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest
40, // 48: milvus.proto.index.IndexCoord.CreateIndex:output_type -> milvus.proto.common.Status
40, // 49: milvus.proto.index.IndexCoord.AlterIndex:output_type -> milvus.proto.common.Status
9, // 50: milvus.proto.index.IndexCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse
12, // 51: milvus.proto.index.IndexCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse
18, // 52: milvus.proto.index.IndexCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse
40, // 53: milvus.proto.index.IndexCoord.DropIndex:output_type -> milvus.proto.common.Status
21, // 54: milvus.proto.index.IndexCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse
28, // 55: milvus.proto.index.IndexCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse
23, // 56: milvus.proto.index.IndexCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse
46, // 57: milvus.proto.index.IndexCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse
47, // 58: milvus.proto.index.IndexCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
48, // 59: milvus.proto.index.IndexCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse
48, // [48:60] is the sub-list for method output_type
36, // [36:48] is the sub-list for method input_type
36, // [36:36] is the sub-list for extension type_name
36, // [36:36] is the sub-list for extension extendee
0, // [0:36] is the sub-list for field type_name
1, // 35: milvus.proto.index.UpdateExternalCollectionTask.state:type_name -> milvus.proto.index.JobState
17, // 36: milvus.proto.index.GetIndexInfoResponse.SegmentInfoEntry.value:type_name -> milvus.proto.index.SegmentInfo
13, // 37: milvus.proto.index.IndexCoord.CreateIndex:input_type -> milvus.proto.index.CreateIndexRequest
14, // 38: milvus.proto.index.IndexCoord.AlterIndex:input_type -> milvus.proto.index.AlterIndexRequest
8, // 39: milvus.proto.index.IndexCoord.GetIndexState:input_type -> milvus.proto.index.GetIndexStateRequest
10, // 40: milvus.proto.index.IndexCoord.GetSegmentIndexState:input_type -> milvus.proto.index.GetSegmentIndexStateRequest
15, // 41: milvus.proto.index.IndexCoord.GetIndexInfos:input_type -> milvus.proto.index.GetIndexInfoRequest
19, // 42: milvus.proto.index.IndexCoord.DropIndex:input_type -> milvus.proto.index.DropIndexRequest
20, // 43: milvus.proto.index.IndexCoord.DescribeIndex:input_type -> milvus.proto.index.DescribeIndexRequest
27, // 44: milvus.proto.index.IndexCoord.GetIndexStatistics:input_type -> milvus.proto.index.GetIndexStatisticsRequest
22, // 45: milvus.proto.index.IndexCoord.GetIndexBuildProgress:input_type -> milvus.proto.index.GetIndexBuildProgressRequest
44, // 46: milvus.proto.index.IndexCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest
45, // 47: milvus.proto.index.IndexCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest
46, // 48: milvus.proto.index.IndexCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest
41, // 49: milvus.proto.index.IndexCoord.CreateIndex:output_type -> milvus.proto.common.Status
41, // 50: milvus.proto.index.IndexCoord.AlterIndex:output_type -> milvus.proto.common.Status
9, // 51: milvus.proto.index.IndexCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse
12, // 52: milvus.proto.index.IndexCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse
18, // 53: milvus.proto.index.IndexCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse
41, // 54: milvus.proto.index.IndexCoord.DropIndex:output_type -> milvus.proto.common.Status
21, // 55: milvus.proto.index.IndexCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse
28, // 56: milvus.proto.index.IndexCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse
23, // 57: milvus.proto.index.IndexCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse
47, // 58: milvus.proto.index.IndexCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse
48, // 59: milvus.proto.index.IndexCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
49, // 60: milvus.proto.index.IndexCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse
49, // [49:61] is the sub-list for method output_type
37, // [37:49] is the sub-list for method input_type
37, // [37:37] is the sub-list for extension type_name
37, // [37:37] is the sub-list for extension extendee
0, // [0:37] is the sub-list for field type_name
}
func init() { file_index_coord_proto_init() }
@ -3936,6 +4060,18 @@ func file_index_coord_proto_init() {
return nil
}
}
file_index_coord_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*UpdateExternalCollectionTask); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
}
type x struct{}
out := protoimpl.TypeBuilder{
@ -3943,7 +4079,7 @@ func file_index_coord_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_index_coord_proto_rawDesc,
NumEnums: 3,
NumMessages: 33,
NumMessages: 34,
NumExtensions: 0,
NumServices: 1,
},