mirror of
https://gitee.com/milvus-io/milvus.git
synced 2025-12-07 01:28:27 +08:00
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:
parent
7c9a9c6f7e
commit
b6532d3e44
2
go.mod
2
go.mod
@ -21,7 +21,7 @@ require (
|
|||||||
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0
|
github.com/grpc-ecosystem/go-grpc-middleware v1.3.0
|
||||||
github.com/klauspost/compress v1.17.9
|
github.com/klauspost/compress v1.17.9
|
||||||
github.com/mgutz/ansi v0.0.0-20200706080929-d51e80ef957d
|
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/minio/minio-go/v7 v7.0.73
|
||||||
github.com/panjf2000/ants/v2 v2.11.3 // indirect
|
github.com/panjf2000/ants/v2 v2.11.3 // indirect
|
||||||
github.com/pingcap/log v1.1.1-0.20221015072633-39906604fb81 // indirect
|
github.com/pingcap/log v1.1.1-0.20221015072633-39906604fb81 // indirect
|
||||||
|
|||||||
4
go.sum
4
go.sum
@ -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/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 h1:TfeY0NxYxZzUfIfYe5qYDBzt4ZYRqzUjTR6CvUzjat8=
|
||||||
github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b/go.mod h1:iwW+9cWfIzzDseEBCCeDSN5SD16Tidvy8cwQ7ZY8Qj4=
|
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.20251119072500-4bd276fe335e h1:NZx+z8JDXrLf/y+bIs360ypdPDM9AZTkkA0Fi1v3MWc=
|
||||||
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/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 h1:AMFGa4R4MiIpspGNG7Z948v4n35fFGB3RR3G/ry4FWs=
|
||||||
github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8/go.mod h1:mC1jAcsrzbxHt8iiaC+zU4b1ylILSosueou12R++wfY=
|
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=
|
github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 h1:+n/aFZefKZp7spd8DFdX7uMikMLXX4oubIzJF4kv/wI=
|
||||||
|
|||||||
@ -18,6 +18,7 @@ import (
|
|||||||
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
"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/milvuspb"
|
||||||
|
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||||
"github.com/milvus-io/milvus/internal/datacoord"
|
"github.com/milvus-io/milvus/internal/datacoord"
|
||||||
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
|
||||||
"github.com/milvus-io/milvus/internal/kv/tikv"
|
"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) {
|
func (s *mixCoordImpl) ListFileResources(ctx context.Context, req *milvuspb.ListFileResourcesRequest) (*milvuspb.ListFileResourcesResponse, error) {
|
||||||
return s.datacoordServer.ListFileResources(ctx, req)
|
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)
|
||||||
|
}
|
||||||
|
|||||||
265
internal/datacoord/external_collection_inspector.go
Normal file
265
internal/datacoord/external_collection_inspector.go
Normal 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
|
||||||
|
}
|
||||||
221
internal/datacoord/external_collection_task_meta.go
Normal file
221
internal/datacoord/external_collection_task_meta.go
Normal 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
|
||||||
|
}
|
||||||
@ -99,6 +99,7 @@ type meta struct {
|
|||||||
partitionStatsMeta *partitionStatsMeta
|
partitionStatsMeta *partitionStatsMeta
|
||||||
compactionTaskMeta *compactionTaskMeta
|
compactionTaskMeta *compactionTaskMeta
|
||||||
statsTaskMeta *statsTaskMeta
|
statsTaskMeta *statsTaskMeta
|
||||||
|
externalCollectionTaskMeta *externalCollectionTaskMeta
|
||||||
|
|
||||||
// File Resource Meta
|
// File Resource Meta
|
||||||
resourceMeta map[string]*model.FileResource
|
resourceMeta map[string]*model.FileResource
|
||||||
@ -183,6 +184,13 @@ func newMeta(ctx context.Context, catalog metastore.DataCoordCatalog, chunkManag
|
|||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// TODO: add external collection task meta
|
||||||
|
// ectm, err := newExternalCollectionTaskMeta(ctx, catalog)
|
||||||
|
// if err != nil {
|
||||||
|
// return nil, err
|
||||||
|
// }
|
||||||
|
|
||||||
mt := &meta{
|
mt := &meta{
|
||||||
ctx: ctx,
|
ctx: ctx,
|
||||||
catalog: catalog,
|
catalog: catalog,
|
||||||
@ -195,6 +203,7 @@ func newMeta(ctx context.Context, catalog metastore.DataCoordCatalog, chunkManag
|
|||||||
partitionStatsMeta: psm,
|
partitionStatsMeta: psm,
|
||||||
compactionTaskMeta: ctm,
|
compactionTaskMeta: ctm,
|
||||||
statsTaskMeta: stm,
|
statsTaskMeta: stm,
|
||||||
|
// externalCollectionTaskMeta: ectm,
|
||||||
resourceMeta: make(map[string]*model.FileResource),
|
resourceMeta: make(map[string]*model.FileResource),
|
||||||
}
|
}
|
||||||
err = mt.reloadFromKV(ctx, broker)
|
err = mt.reloadFromKV(ctx, broker)
|
||||||
|
|||||||
@ -28,6 +28,7 @@ import (
|
|||||||
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
"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/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-proto/go-api/v2/schemapb"
|
||||||
"github.com/milvus-io/milvus/internal/datacoord/allocator"
|
"github.com/milvus-io/milvus/internal/datacoord/allocator"
|
||||||
"github.com/milvus-io/milvus/internal/datacoord/broker"
|
"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")
|
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 {
|
func newMockMixCoord() *mockMixCoord {
|
||||||
return &mockMixCoord{state: commonpb.StateCode_Healthy}
|
return &mockMixCoord{state: commonpb.StateCode_Healthy}
|
||||||
}
|
}
|
||||||
|
|||||||
@ -159,6 +159,7 @@ type Server struct {
|
|||||||
statsInspector *statsInspector
|
statsInspector *statsInspector
|
||||||
indexInspector *indexInspector
|
indexInspector *indexInspector
|
||||||
analyzeInspector *analyzeInspector
|
analyzeInspector *analyzeInspector
|
||||||
|
externalCollectionInspector *externalCollectionInspector
|
||||||
globalScheduler task.GlobalScheduler
|
globalScheduler task.GlobalScheduler
|
||||||
|
|
||||||
// manage ways that data coord access other coord
|
// manage ways that data coord access other coord
|
||||||
@ -321,6 +322,10 @@ func (s *Server) initDataCoord() error {
|
|||||||
s.initStatsInspector()
|
s.initStatsInspector()
|
||||||
log.Info("init statsJobManager done")
|
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 {
|
if err = s.initSegmentManager(); err != nil {
|
||||||
return err
|
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() {
|
func (s *Server) initCompaction() {
|
||||||
cph := newCompactionInspector(s.meta, s.allocator, s.handler, s.globalScheduler, s.indexEngineVersionManager)
|
cph := newCompactionInspector(s.meta, s.allocator, s.handler, s.globalScheduler, s.indexEngineVersionManager)
|
||||||
cph.loadMeta()
|
cph.loadMeta()
|
||||||
@ -749,6 +760,8 @@ func (s *Server) startTaskScheduler() {
|
|||||||
s.statsInspector.Start()
|
s.statsInspector.Start()
|
||||||
s.indexInspector.Start()
|
s.indexInspector.Start()
|
||||||
s.analyzeInspector.Start()
|
s.analyzeInspector.Start()
|
||||||
|
// TODO: enable external collection inspector
|
||||||
|
// s.externalCollectionInspector.Start()
|
||||||
s.startCollectMetaMetrics(s.serverLoopCtx)
|
s.startCollectMetaMetrics(s.serverLoopCtx)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1060,6 +1073,9 @@ func (s *Server) Stop() error {
|
|||||||
if s.qnSessionWatcher != nil {
|
if s.qnSessionWatcher != nil {
|
||||||
s.qnSessionWatcher.Stop()
|
s.qnSessionWatcher.Stop()
|
||||||
}
|
}
|
||||||
|
// TODO: enable external collection inspector
|
||||||
|
// s.externalCollectionInspector.Stop()
|
||||||
|
// log.Info("datacoord external collection inspector stopped")
|
||||||
|
|
||||||
if s.session != nil {
|
if s.session != nil {
|
||||||
s.session.Stop()
|
s.session.Stop()
|
||||||
|
|||||||
@ -2098,3 +2098,48 @@ func (s *Server) ListFileResources(ctx context.Context, req *milvuspb.ListFileRe
|
|||||||
Resources: fileResources,
|
Resources: fileResources,
|
||||||
}, nil
|
}, 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
|
||||||
|
}
|
||||||
|
|||||||
149
internal/datacoord/task_update_external_collection.go
Normal file
149
internal/datacoord/task_update_external_collection.go
Normal 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
|
||||||
|
}
|
||||||
@ -27,6 +27,7 @@ import (
|
|||||||
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
"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/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/distributed/utils"
|
||||||
"github.com/milvus-io/milvus/internal/types"
|
"github.com/milvus-io/milvus/internal/types"
|
||||||
"github.com/milvus-io/milvus/internal/util/grpcclient"
|
"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)
|
// AssignSegmentID applies allocations for specified Coolection/Partition and related Channel Name(Virtial Channel)
|
||||||
//
|
//
|
||||||
// ctx is the context to control request deadline and cancellation
|
// ctx is the context to control request deadline and cancellation
|
||||||
|
|||||||
@ -31,6 +31,7 @@ import (
|
|||||||
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
"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/milvuspb"
|
||||||
|
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
||||||
mixcoord "github.com/milvus-io/milvus/internal/coordinator"
|
mixcoord "github.com/milvus-io/milvus/internal/coordinator"
|
||||||
mix "github.com/milvus-io/milvus/internal/distributed/mixcoord/client"
|
mix "github.com/milvus-io/milvus/internal/distributed/mixcoord/client"
|
||||||
"github.com/milvus-io/milvus/internal/distributed/utils"
|
"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)
|
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
|
// AssignSegmentID requests to allocate segment space for insert
|
||||||
func (s *Server) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error) {
|
func (s *Server) AssignSegmentID(ctx context.Context, req *datapb.AssignSegmentIDRequest) (*datapb.AssignSegmentIDResponse, error) {
|
||||||
return s.mixCoord.AssignSegmentID(ctx, req)
|
return s.mixCoord.AssignSegmentID(ctx, req)
|
||||||
|
|||||||
@ -180,6 +180,10 @@ type DataCoordCatalog interface {
|
|||||||
SaveStatsTask(ctx context.Context, task *indexpb.StatsTask) error
|
SaveStatsTask(ctx context.Context, task *indexpb.StatsTask) error
|
||||||
DropStatsTask(ctx context.Context, taskID typeutil.UniqueID) 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
|
// Analyzer Resource
|
||||||
SaveFileResource(ctx context.Context, resource *model.FileResource) error
|
SaveFileResource(ctx context.Context, resource *model.FileResource) error
|
||||||
RemoveFileResource(ctx context.Context, resourceID int64) error
|
RemoveFileResource(ctx context.Context, resourceID int64) error
|
||||||
|
|||||||
@ -33,6 +33,7 @@ const (
|
|||||||
PartitionStatsInfoPrefix = MetaPrefix + "/partition-stats"
|
PartitionStatsInfoPrefix = MetaPrefix + "/partition-stats"
|
||||||
PartitionStatsCurrentVersionPrefix = MetaPrefix + "/current-partition-stats-version"
|
PartitionStatsCurrentVersionPrefix = MetaPrefix + "/current-partition-stats-version"
|
||||||
StatsTaskPrefix = MetaPrefix + "/stats-task"
|
StatsTaskPrefix = MetaPrefix + "/stats-task"
|
||||||
|
UpdateExternalCollectionTaskPrefix = MetaPrefix + "/update-external-collection-task"
|
||||||
FileResourceMetaPrefix = MetaPrefix + "/file_resource"
|
FileResourceMetaPrefix = MetaPrefix + "/file_resource"
|
||||||
|
|
||||||
NonRemoveFlagTomestone = "non-removed"
|
NonRemoveFlagTomestone = "non-removed"
|
||||||
|
|||||||
@ -963,6 +963,45 @@ func (kc *Catalog) DropStatsTask(ctx context.Context, taskID typeutil.UniqueID)
|
|||||||
return kc.MetaKv.Remove(ctx, key)
|
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 {
|
func (kc *Catalog) SaveFileResource(ctx context.Context, resource *model.FileResource) error {
|
||||||
k := BuildFileResourceKey(resource.ID)
|
k := BuildFileResourceKey(resource.ID)
|
||||||
v, err := proto.Marshal(resource.Marshal())
|
v, err := proto.Marshal(resource.Marshal())
|
||||||
|
|||||||
@ -352,3 +352,7 @@ func buildAnalyzeTaskKey(taskID int64) string {
|
|||||||
func buildStatsTaskKey(taskID int64) string {
|
func buildStatsTaskKey(taskID int64) string {
|
||||||
return fmt.Sprintf("%s/%d", StatsTaskPrefix, taskID)
|
return fmt.Sprintf("%s/%d", StatsTaskPrefix, taskID)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func buildUpdateExternalCollectionTaskKey(taskID int64) string {
|
||||||
|
return fmt.Sprintf("%s/%d", UpdateExternalCollectionTaskPrefix, taskID)
|
||||||
|
}
|
||||||
|
|||||||
@ -991,6 +991,53 @@ func (_c *DataCoordCatalog_DropStatsTask_Call) RunAndReturn(run func(context.Con
|
|||||||
return _c
|
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
|
// GcConfirm provides a mock function with given fields: ctx, collectionID, partitionID
|
||||||
func (_m *DataCoordCatalog) GcConfirm(ctx context.Context, collectionID int64, partitionID int64) bool {
|
func (_m *DataCoordCatalog) GcConfirm(ctx context.Context, collectionID int64, partitionID int64) bool {
|
||||||
ret := _m.Called(ctx, collectionID, partitionID)
|
ret := _m.Called(ctx, collectionID, partitionID)
|
||||||
@ -1795,6 +1842,64 @@ func (_c *DataCoordCatalog_ListStatsTasks_Call) RunAndReturn(run func(context.Co
|
|||||||
return _c
|
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
|
// MarkChannelAdded provides a mock function with given fields: ctx, channel
|
||||||
func (_m *DataCoordCatalog) MarkChannelAdded(ctx context.Context, channel string) error {
|
func (_m *DataCoordCatalog) MarkChannelAdded(ctx context.Context, channel string) error {
|
||||||
ret := _m.Called(ctx, channel)
|
ret := _m.Called(ctx, channel)
|
||||||
@ -2504,6 +2609,53 @@ func (_c *DataCoordCatalog_SaveStatsTask_Call) RunAndReturn(run func(context.Con
|
|||||||
return _c
|
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
|
// ShouldDropChannel provides a mock function with given fields: ctx, channel
|
||||||
func (_m *DataCoordCatalog) ShouldDropChannel(ctx context.Context, channel string) bool {
|
func (_m *DataCoordCatalog) ShouldDropChannel(ctx context.Context, channel string) bool {
|
||||||
ret := _m.Called(ctx, channel)
|
ret := _m.Called(ctx, channel)
|
||||||
|
|||||||
@ -18,6 +18,8 @@ import (
|
|||||||
|
|
||||||
mock "github.com/stretchr/testify/mock"
|
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"
|
txnkv "github.com/tikv/client-go/v2/txnkv"
|
||||||
|
|
||||||
types "github.com/milvus-io/milvus/internal/types"
|
types "github.com/milvus-io/milvus/internal/types"
|
||||||
@ -390,6 +392,65 @@ func (_c *MockDataCoord_CheckHealth_Call) RunAndReturn(run func(context.Context,
|
|||||||
return _c
|
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
|
// CreateIndex provides a mock function with given fields: _a0, _a1
|
||||||
func (_m *MockDataCoord) CreateIndex(_a0 context.Context, _a1 *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
|
func (_m *MockDataCoord) CreateIndex(_a0 context.Context, _a1 *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
|
||||||
ret := _m.Called(_a0, _a1)
|
ret := _m.Called(_a0, _a1)
|
||||||
|
|||||||
@ -18,6 +18,8 @@ import (
|
|||||||
milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||||
|
|
||||||
mock "github.com/stretchr/testify/mock"
|
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
|
// 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
|
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
|
// 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) {
|
func (_m *MockDataCoordClient) CreateIndex(ctx context.Context, in *indexpb.CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||||
_va := make([]interface{}, len(opts))
|
_va := make([]interface{}, len(opts))
|
||||||
|
|||||||
@ -20,6 +20,8 @@ import (
|
|||||||
|
|
||||||
mock "github.com/stretchr/testify/mock"
|
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"
|
proxypb "github.com/milvus-io/milvus/pkg/v2/proto/proxypb"
|
||||||
|
|
||||||
querypb "github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
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
|
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
|
// CreateIndex provides a mock function with given fields: _a0, _a1
|
||||||
func (_m *MixCoord) CreateIndex(_a0 context.Context, _a1 *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
|
func (_m *MixCoord) CreateIndex(_a0 context.Context, _a1 *indexpb.CreateIndexRequest) (*commonpb.Status, error) {
|
||||||
ret := _m.Called(_a0, _a1)
|
ret := _m.Called(_a0, _a1)
|
||||||
|
|||||||
@ -19,6 +19,8 @@ import (
|
|||||||
|
|
||||||
mock "github.com/stretchr/testify/mock"
|
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"
|
proxypb "github.com/milvus-io/milvus/pkg/v2/proto/proxypb"
|
||||||
|
|
||||||
querypb "github.com/milvus-io/milvus/pkg/v2/proto/querypb"
|
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
|
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
|
// 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) {
|
func (_m *MockMixCoordClient) CreateIndex(ctx context.Context, in *indexpb.CreateIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||||
_va := make([]interface{}, len(opts))
|
_va := make([]interface{}, len(opts))
|
||||||
|
|||||||
@ -476,6 +476,52 @@ func (_c *MockCSegment_Insert_Call) RunAndReturn(run func(context.Context, *segc
|
|||||||
return _c
|
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
|
// LoadFieldData provides a mock function with given fields: ctx, request
|
||||||
func (_m *MockCSegment) LoadFieldData(ctx context.Context, request *segcore.LoadFieldDataRequest) (*segcore.LoadFieldDataResult, error) {
|
func (_m *MockCSegment) LoadFieldData(ctx context.Context, request *segcore.LoadFieldDataRequest) (*segcore.LoadFieldDataResult, error) {
|
||||||
ret := _m.Called(ctx, request)
|
ret := _m.Called(ctx, request)
|
||||||
|
|||||||
@ -29,6 +29,7 @@ import (
|
|||||||
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
"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/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-proto/go-api/v2/schemapb"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/common"
|
"github.com/milvus-io/milvus/pkg/v2/common"
|
||||||
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
|
"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
|
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 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)
|
type ShowPartitionsFunc func(ctx context.Context, request *milvuspb.ShowPartitionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowPartitionsResponse, error)
|
||||||
|
|||||||
@ -32,6 +32,8 @@ service DataCoord {
|
|||||||
rpc Flush(FlushRequest) returns (FlushResponse) {}
|
rpc Flush(FlushRequest) returns (FlushResponse) {}
|
||||||
rpc FlushAll(FlushAllRequest) returns(FlushAllResponse) {}
|
rpc FlushAll(FlushAllRequest) returns(FlushAllResponse) {}
|
||||||
|
|
||||||
|
rpc CreateExternalCollection(msg.CreateCollectionRequest) returns (CreateExternalCollectionResponse) {}
|
||||||
|
|
||||||
|
|
||||||
// AllocSegment alloc a new growing segment, add it into segment meta.
|
// AllocSegment alloc a new growing segment, add it into segment meta.
|
||||||
rpc AllocSegment(AllocSegmentRequest) returns (AllocSegmentResponse) {}
|
rpc AllocSegment(AllocSegmentRequest) returns (AllocSegmentResponse) {}
|
||||||
@ -1137,3 +1139,7 @@ message FileResourceInfo {
|
|||||||
string path = 2;
|
string path = 2;
|
||||||
int64 resource_id = 3;
|
int64 resource_id = 3;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
message CreateExternalCollectionResponse {
|
||||||
|
common.Status status = 1;
|
||||||
|
}
|
||||||
File diff suppressed because it is too large
Load Diff
@ -10,6 +10,7 @@ import (
|
|||||||
context "context"
|
context "context"
|
||||||
commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||||
milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
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"
|
indexpb "github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
|
||||||
internalpb "github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
internalpb "github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
|
||||||
grpc "google.golang.org/grpc"
|
grpc "google.golang.org/grpc"
|
||||||
@ -25,6 +26,7 @@ const _ = grpc.SupportPackageIsVersion7
|
|||||||
const (
|
const (
|
||||||
DataCoord_Flush_FullMethodName = "/milvus.proto.data.DataCoord/Flush"
|
DataCoord_Flush_FullMethodName = "/milvus.proto.data.DataCoord/Flush"
|
||||||
DataCoord_FlushAll_FullMethodName = "/milvus.proto.data.DataCoord/FlushAll"
|
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_AllocSegment_FullMethodName = "/milvus.proto.data.DataCoord/AllocSegment"
|
||||||
DataCoord_AssignSegmentID_FullMethodName = "/milvus.proto.data.DataCoord/AssignSegmentID"
|
DataCoord_AssignSegmentID_FullMethodName = "/milvus.proto.data.DataCoord/AssignSegmentID"
|
||||||
DataCoord_GetSegmentInfo_FullMethodName = "/milvus.proto.data.DataCoord/GetSegmentInfo"
|
DataCoord_GetSegmentInfo_FullMethodName = "/milvus.proto.data.DataCoord/GetSegmentInfo"
|
||||||
@ -81,6 +83,7 @@ const (
|
|||||||
type DataCoordClient interface {
|
type DataCoordClient interface {
|
||||||
Flush(ctx context.Context, in *FlushRequest, opts ...grpc.CallOption) (*FlushResponse, error)
|
Flush(ctx context.Context, in *FlushRequest, opts ...grpc.CallOption) (*FlushResponse, error)
|
||||||
FlushAll(ctx context.Context, in *FlushAllRequest, opts ...grpc.CallOption) (*FlushAllResponse, 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 alloc a new growing segment, add it into segment meta.
|
||||||
AllocSegment(ctx context.Context, in *AllocSegmentRequest, opts ...grpc.CallOption) (*AllocSegmentResponse, error)
|
AllocSegment(ctx context.Context, in *AllocSegmentRequest, opts ...grpc.CallOption) (*AllocSegmentResponse, error)
|
||||||
// Deprecated: Do not use.
|
// Deprecated: Do not use.
|
||||||
@ -166,6 +169,15 @@ func (c *dataCoordClient) FlushAll(ctx context.Context, in *FlushAllRequest, opt
|
|||||||
return out, nil
|
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) {
|
func (c *dataCoordClient) AllocSegment(ctx context.Context, in *AllocSegmentRequest, opts ...grpc.CallOption) (*AllocSegmentResponse, error) {
|
||||||
out := new(AllocSegmentResponse)
|
out := new(AllocSegmentResponse)
|
||||||
err := c.cc.Invoke(ctx, DataCoord_AllocSegment_FullMethodName, in, out, opts...)
|
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 {
|
type DataCoordServer interface {
|
||||||
Flush(context.Context, *FlushRequest) (*FlushResponse, error)
|
Flush(context.Context, *FlushRequest) (*FlushResponse, error)
|
||||||
FlushAll(context.Context, *FlushAllRequest) (*FlushAllResponse, 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 alloc a new growing segment, add it into segment meta.
|
||||||
AllocSegment(context.Context, *AllocSegmentRequest) (*AllocSegmentResponse, error)
|
AllocSegment(context.Context, *AllocSegmentRequest) (*AllocSegmentResponse, error)
|
||||||
// Deprecated: Do not use.
|
// Deprecated: Do not use.
|
||||||
@ -675,6 +688,9 @@ func (UnimplementedDataCoordServer) Flush(context.Context, *FlushRequest) (*Flus
|
|||||||
func (UnimplementedDataCoordServer) FlushAll(context.Context, *FlushAllRequest) (*FlushAllResponse, error) {
|
func (UnimplementedDataCoordServer) FlushAll(context.Context, *FlushAllRequest) (*FlushAllResponse, error) {
|
||||||
return nil, status.Errorf(codes.Unimplemented, "method FlushAll not implemented")
|
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) {
|
func (UnimplementedDataCoordServer) AllocSegment(context.Context, *AllocSegmentRequest) (*AllocSegmentResponse, error) {
|
||||||
return nil, status.Errorf(codes.Unimplemented, "method AllocSegment not implemented")
|
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)
|
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) {
|
func _DataCoord_AllocSegment_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||||
in := new(AllocSegmentRequest)
|
in := new(AllocSegmentRequest)
|
||||||
if err := dec(in); err != nil {
|
if err := dec(in); err != nil {
|
||||||
@ -1746,6 +1780,10 @@ var DataCoord_ServiceDesc = grpc.ServiceDesc{
|
|||||||
MethodName: "FlushAll",
|
MethodName: "FlushAll",
|
||||||
Handler: _DataCoord_FlushAll_Handler,
|
Handler: _DataCoord_FlushAll_Handler,
|
||||||
},
|
},
|
||||||
|
{
|
||||||
|
MethodName: "CreateExternalCollection",
|
||||||
|
Handler: _DataCoord_CreateExternalCollection_Handler,
|
||||||
|
},
|
||||||
{
|
{
|
||||||
MethodName: "AllocSegment",
|
MethodName: "AllocSegment",
|
||||||
Handler: _DataCoord_AllocSegment_Handler,
|
Handler: _DataCoord_AllocSegment_Handler,
|
||||||
|
|||||||
@ -325,6 +325,17 @@ message StatsTask {
|
|||||||
bool canRecycle = 12;
|
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 {
|
enum StatsSubJob {
|
||||||
None = 0;
|
None = 0;
|
||||||
// deprecated, after v2.6.0, move to compaction
|
// deprecated, after v2.6.0, move to compaction
|
||||||
|
|||||||
@ -2827,6 +2827,109 @@ func (x *StatsTask) GetCanRecycle() bool {
|
|||||||
return false
|
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 protoreflect.FileDescriptor
|
||||||
|
|
||||||
var file_index_coord_proto_rawDesc = []byte{
|
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,
|
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,
|
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,
|
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,
|
0x65, 0x22, 0xaf, 0x02, 0x0a, 0x1c, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x78, 0x74, 0x65,
|
||||||
0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x13, 0x0a,
|
0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61,
|
||||||
0x0f, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62,
|
0x73, 0x6b, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
|
||||||
0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x41, 0x6e, 0x61,
|
0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
|
||||||
0x6c, 0x79, 0x7a, 0x65, 0x4a, 0x6f, 0x62, 0x10, 0x02, 0x12, 0x13, 0x0a, 0x0f, 0x4a, 0x6f, 0x62,
|
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44,
|
||||||
0x54, 0x79, 0x70, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4a, 0x6f, 0x62, 0x10, 0x03, 0x2a, 0x83,
|
0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x18,
|
||||||
0x01, 0x0a, 0x08, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x10, 0x0a, 0x0c, 0x4a,
|
0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52,
|
||||||
0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x10, 0x0a,
|
0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65,
|
||||||
0x0c, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x69, 0x74, 0x10, 0x01, 0x12,
|
0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44,
|
||||||
0x16, 0x0a, 0x12, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x50, 0x72, 0x6f,
|
0x12, 0x32, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32,
|
||||||
0x67, 0x72, 0x65, 0x73, 0x73, 0x10, 0x02, 0x12, 0x14, 0x0a, 0x10, 0x4a, 0x6f, 0x62, 0x53, 0x74,
|
0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69,
|
||||||
0x61, 0x74, 0x65, 0x46, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x10, 0x03, 0x12, 0x12, 0x0a,
|
0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73,
|
||||||
0x0e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10,
|
0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61,
|
||||||
0x04, 0x12, 0x11, 0x0a, 0x0d, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x74,
|
0x73, 0x6f, 0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52,
|
||||||
0x72, 0x79, 0x10, 0x05, 0x2a, 0x55, 0x0a, 0x0b, 0x53, 0x74, 0x61, 0x74, 0x73, 0x53, 0x75, 0x62,
|
0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x27, 0x0a, 0x0f, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61,
|
||||||
0x4a, 0x6f, 0x62, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x08, 0x0a,
|
0x6c, 0x5f, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e,
|
||||||
0x04, 0x53, 0x6f, 0x72, 0x74, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x65, 0x78, 0x74, 0x49,
|
0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x23,
|
||||||
0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x42, 0x4d, 0x32,
|
0x0a, 0x0d, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18,
|
||||||
0x35, 0x4a, 0x6f, 0x62, 0x10, 0x03, 0x12, 0x13, 0x0a, 0x0f, 0x4a, 0x73, 0x6f, 0x6e, 0x4b, 0x65,
|
0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x53,
|
||||||
0x79, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a, 0x6f, 0x62, 0x10, 0x04, 0x32, 0xf4, 0x09, 0x0a, 0x0a,
|
0x70, 0x65, 0x63, 0x2a, 0x5b, 0x0a, 0x07, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f,
|
||||||
0x49, 0x6e, 0x64, 0x65, 0x78, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x12, 0x54, 0x0a, 0x0b, 0x43, 0x72,
|
0x0a, 0x0b, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12,
|
||||||
0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
0x13, 0x0a, 0x0f, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x4a,
|
||||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43,
|
0x6f, 0x62, 0x10, 0x01, 0x12, 0x15, 0x0a, 0x11, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x41,
|
||||||
0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
|
0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x4a, 0x6f, 0x62, 0x10, 0x02, 0x12, 0x13, 0x0a, 0x0f, 0x4a,
|
||||||
0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
|
0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x4a, 0x6f, 0x62, 0x10, 0x03,
|
||||||
0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00,
|
0x2a, 0x83, 0x01, 0x0a, 0x08, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x10, 0x0a,
|
||||||
0x12, 0x52, 0x0a, 0x0a, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x25,
|
0x0c, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12,
|
||||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e,
|
0x10, 0x0a, 0x0c, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x69, 0x74, 0x10,
|
||||||
0x64, 0x65, 0x78, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65,
|
0x01, 0x12, 0x16, 0x0a, 0x12, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x50,
|
||||||
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x10, 0x02, 0x12, 0x14, 0x0a, 0x10, 0x4a, 0x6f, 0x62,
|
||||||
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74,
|
0x53, 0x74, 0x61, 0x74, 0x65, 0x46, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x65, 0x64, 0x10, 0x03, 0x12,
|
||||||
0x75, 0x73, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78,
|
0x12, 0x0a, 0x0e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x46, 0x61, 0x69, 0x6c, 0x65,
|
||||||
0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
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,
|
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,
|
0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
|
||||||
0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69,
|
0x00, 0x12, 0x50, 0x0a, 0x09, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x24,
|
||||||
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,
|
|
||||||
0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e,
|
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,
|
0x64, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71,
|
||||||
0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
|
0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
|
||||||
0x12, 0x7e, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c,
|
0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75,
|
||||||
0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76,
|
0x73, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49,
|
||||||
0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47,
|
0x6e, 0x64, 0x65, 0x78, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72,
|
||||||
0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67,
|
0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69,
|
||||||
0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 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,
|
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,
|
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,
|
0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e,
|
||||||
0x12, 0x7b, 0x0a, 0x12, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72,
|
0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64,
|
||||||
0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64,
|
||||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53,
|
0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
|
||||||
0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e,
|
0x22, 0x00, 0x12, 0x7b, 0x0a, 0x12, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67,
|
||||||
0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
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,
|
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,
|
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,
|
0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c,
|
||||||
0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69,
|
0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e,
|
||||||
0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61,
|
||||||
0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75,
|
0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
|
||||||
0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f,
|
0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x26, 0x2e,
|
||||||
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,
|
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,
|
0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65,
|
||||||
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e,
|
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70,
|
||||||
0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65,
|
0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d,
|
||||||
0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
|
0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
|
||||||
0x22, 0x00, 0x42, 0x32, 0x5a, 0x30, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d,
|
0x12, 0x62, 0x0a, 0x0b, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12,
|
||||||
0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75,
|
0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d,
|
||||||
0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x69,
|
0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74,
|
||||||
0x6e, 0x64, 0x65, 0x78, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
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 (
|
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_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{}{
|
var file_index_coord_proto_goTypes = []interface{}{
|
||||||
(JobType)(0), // 0: milvus.proto.index.JobType
|
(JobType)(0), // 0: milvus.proto.index.JobType
|
||||||
(JobState)(0), // 1: milvus.proto.index.JobState
|
(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
|
(*SegmentStats)(nil), // 32: milvus.proto.index.SegmentStats
|
||||||
(*FieldLogPath)(nil), // 33: milvus.proto.index.FieldLogPath
|
(*FieldLogPath)(nil), // 33: milvus.proto.index.FieldLogPath
|
||||||
(*StatsTask)(nil), // 34: milvus.proto.index.StatsTask
|
(*StatsTask)(nil), // 34: milvus.proto.index.StatsTask
|
||||||
nil, // 35: milvus.proto.index.GetIndexInfoResponse.SegmentInfoEntry
|
(*UpdateExternalCollectionTask)(nil), // 35: milvus.proto.index.UpdateExternalCollectionTask
|
||||||
(*commonpb.KeyValuePair)(nil), // 36: milvus.proto.common.KeyValuePair
|
nil, // 36: milvus.proto.index.GetIndexInfoResponse.SegmentInfoEntry
|
||||||
(commonpb.IndexState)(0), // 37: milvus.proto.common.IndexState
|
(*commonpb.KeyValuePair)(nil), // 37: milvus.proto.common.KeyValuePair
|
||||||
(*commonpb.MsgBase)(nil), // 38: milvus.proto.common.MsgBase
|
(commonpb.IndexState)(0), // 38: milvus.proto.common.IndexState
|
||||||
(*commonpb.Address)(nil), // 39: milvus.proto.common.Address
|
(*commonpb.MsgBase)(nil), // 39: milvus.proto.common.MsgBase
|
||||||
(*commonpb.Status)(nil), // 40: milvus.proto.common.Status
|
(*commonpb.Address)(nil), // 40: milvus.proto.common.Address
|
||||||
(*internalpb.InitParams)(nil), // 41: milvus.proto.internal.InitParams
|
(*commonpb.Status)(nil), // 41: milvus.proto.common.Status
|
||||||
(schemapb.DataType)(0), // 42: milvus.proto.schema.DataType
|
(*internalpb.InitParams)(nil), // 42: milvus.proto.internal.InitParams
|
||||||
(*internalpb.ShowConfigurationsRequest)(nil), // 43: milvus.proto.internal.ShowConfigurationsRequest
|
(schemapb.DataType)(0), // 43: milvus.proto.schema.DataType
|
||||||
(*milvuspb.GetMetricsRequest)(nil), // 44: milvus.proto.milvus.GetMetricsRequest
|
(*internalpb.ShowConfigurationsRequest)(nil), // 44: milvus.proto.internal.ShowConfigurationsRequest
|
||||||
(*milvuspb.CheckHealthRequest)(nil), // 45: milvus.proto.milvus.CheckHealthRequest
|
(*milvuspb.GetMetricsRequest)(nil), // 45: milvus.proto.milvus.GetMetricsRequest
|
||||||
(*internalpb.ShowConfigurationsResponse)(nil), // 46: milvus.proto.internal.ShowConfigurationsResponse
|
(*milvuspb.CheckHealthRequest)(nil), // 46: milvus.proto.milvus.CheckHealthRequest
|
||||||
(*milvuspb.GetMetricsResponse)(nil), // 47: milvus.proto.milvus.GetMetricsResponse
|
(*internalpb.ShowConfigurationsResponse)(nil), // 47: milvus.proto.internal.ShowConfigurationsResponse
|
||||||
(*milvuspb.CheckHealthResponse)(nil), // 48: milvus.proto.milvus.CheckHealthResponse
|
(*milvuspb.GetMetricsResponse)(nil), // 48: milvus.proto.milvus.GetMetricsResponse
|
||||||
|
(*milvuspb.CheckHealthResponse)(nil), // 49: milvus.proto.milvus.CheckHealthResponse
|
||||||
}
|
}
|
||||||
var file_index_coord_proto_depIdxs = []int32{
|
var file_index_coord_proto_depIdxs = []int32{
|
||||||
36, // 0: milvus.proto.index.IndexInfo.type_params:type_name -> milvus.proto.common.KeyValuePair
|
37, // 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, // 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
|
38, // 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, // 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
|
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, // 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, // 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, // 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, // 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
|
42, // 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
|
41, // 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
|
38, // 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
|
38, // 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
|
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
|
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
|
37, // 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
|
37, // 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
|
37, // 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
|
37, // 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, // 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
|
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
|
41, // 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
|
36, // 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, // 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
|
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
|
41, // 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
|
37, // 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, // 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
|
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
|
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, // 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
|
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
|
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
|
1, // 35: milvus.proto.index.UpdateExternalCollectionTask.state:type_name -> milvus.proto.index.JobState
|
||||||
13, // 36: milvus.proto.index.IndexCoord.CreateIndex:input_type -> milvus.proto.index.CreateIndexRequest
|
17, // 36: milvus.proto.index.GetIndexInfoResponse.SegmentInfoEntry.value:type_name -> milvus.proto.index.SegmentInfo
|
||||||
14, // 37: milvus.proto.index.IndexCoord.AlterIndex:input_type -> milvus.proto.index.AlterIndexRequest
|
13, // 37: milvus.proto.index.IndexCoord.CreateIndex:input_type -> milvus.proto.index.CreateIndexRequest
|
||||||
8, // 38: milvus.proto.index.IndexCoord.GetIndexState:input_type -> milvus.proto.index.GetIndexStateRequest
|
14, // 38: milvus.proto.index.IndexCoord.AlterIndex:input_type -> milvus.proto.index.AlterIndexRequest
|
||||||
10, // 39: milvus.proto.index.IndexCoord.GetSegmentIndexState:input_type -> milvus.proto.index.GetSegmentIndexStateRequest
|
8, // 39: milvus.proto.index.IndexCoord.GetIndexState:input_type -> milvus.proto.index.GetIndexStateRequest
|
||||||
15, // 40: milvus.proto.index.IndexCoord.GetIndexInfos:input_type -> milvus.proto.index.GetIndexInfoRequest
|
10, // 40: milvus.proto.index.IndexCoord.GetSegmentIndexState:input_type -> milvus.proto.index.GetSegmentIndexStateRequest
|
||||||
19, // 41: milvus.proto.index.IndexCoord.DropIndex:input_type -> milvus.proto.index.DropIndexRequest
|
15, // 41: milvus.proto.index.IndexCoord.GetIndexInfos:input_type -> milvus.proto.index.GetIndexInfoRequest
|
||||||
20, // 42: milvus.proto.index.IndexCoord.DescribeIndex:input_type -> milvus.proto.index.DescribeIndexRequest
|
19, // 42: milvus.proto.index.IndexCoord.DropIndex:input_type -> milvus.proto.index.DropIndexRequest
|
||||||
27, // 43: milvus.proto.index.IndexCoord.GetIndexStatistics:input_type -> milvus.proto.index.GetIndexStatisticsRequest
|
20, // 43: milvus.proto.index.IndexCoord.DescribeIndex:input_type -> milvus.proto.index.DescribeIndexRequest
|
||||||
22, // 44: milvus.proto.index.IndexCoord.GetIndexBuildProgress:input_type -> milvus.proto.index.GetIndexBuildProgressRequest
|
27, // 44: milvus.proto.index.IndexCoord.GetIndexStatistics:input_type -> milvus.proto.index.GetIndexStatisticsRequest
|
||||||
43, // 45: milvus.proto.index.IndexCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest
|
22, // 45: milvus.proto.index.IndexCoord.GetIndexBuildProgress:input_type -> milvus.proto.index.GetIndexBuildProgressRequest
|
||||||
44, // 46: milvus.proto.index.IndexCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest
|
44, // 46: milvus.proto.index.IndexCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest
|
||||||
45, // 47: milvus.proto.index.IndexCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest
|
45, // 47: milvus.proto.index.IndexCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest
|
||||||
40, // 48: milvus.proto.index.IndexCoord.CreateIndex:output_type -> milvus.proto.common.Status
|
46, // 48: milvus.proto.index.IndexCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest
|
||||||
40, // 49: milvus.proto.index.IndexCoord.AlterIndex:output_type -> milvus.proto.common.Status
|
41, // 49: milvus.proto.index.IndexCoord.CreateIndex:output_type -> milvus.proto.common.Status
|
||||||
9, // 50: milvus.proto.index.IndexCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse
|
41, // 50: milvus.proto.index.IndexCoord.AlterIndex:output_type -> milvus.proto.common.Status
|
||||||
12, // 51: milvus.proto.index.IndexCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse
|
9, // 51: milvus.proto.index.IndexCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse
|
||||||
18, // 52: milvus.proto.index.IndexCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse
|
12, // 52: milvus.proto.index.IndexCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse
|
||||||
40, // 53: milvus.proto.index.IndexCoord.DropIndex:output_type -> milvus.proto.common.Status
|
18, // 53: milvus.proto.index.IndexCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse
|
||||||
21, // 54: milvus.proto.index.IndexCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse
|
41, // 54: milvus.proto.index.IndexCoord.DropIndex:output_type -> milvus.proto.common.Status
|
||||||
28, // 55: milvus.proto.index.IndexCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse
|
21, // 55: milvus.proto.index.IndexCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse
|
||||||
23, // 56: milvus.proto.index.IndexCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse
|
28, // 56: milvus.proto.index.IndexCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse
|
||||||
46, // 57: milvus.proto.index.IndexCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse
|
23, // 57: milvus.proto.index.IndexCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse
|
||||||
47, // 58: milvus.proto.index.IndexCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
|
47, // 58: milvus.proto.index.IndexCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse
|
||||||
48, // 59: milvus.proto.index.IndexCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse
|
48, // 59: milvus.proto.index.IndexCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
|
||||||
48, // [48:60] is the sub-list for method output_type
|
49, // 60: milvus.proto.index.IndexCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse
|
||||||
36, // [36:48] is the sub-list for method input_type
|
49, // [49:61] is the sub-list for method output_type
|
||||||
36, // [36:36] is the sub-list for extension type_name
|
37, // [37:49] is the sub-list for method input_type
|
||||||
36, // [36:36] is the sub-list for extension extendee
|
37, // [37:37] is the sub-list for extension type_name
|
||||||
0, // [0:36] is the sub-list for field 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() }
|
func init() { file_index_coord_proto_init() }
|
||||||
@ -3936,6 +4060,18 @@ func file_index_coord_proto_init() {
|
|||||||
return nil
|
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{}
|
type x struct{}
|
||||||
out := protoimpl.TypeBuilder{
|
out := protoimpl.TypeBuilder{
|
||||||
@ -3943,7 +4079,7 @@ func file_index_coord_proto_init() {
|
|||||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||||
RawDescriptor: file_index_coord_proto_rawDesc,
|
RawDescriptor: file_index_coord_proto_rawDesc,
|
||||||
NumEnums: 3,
|
NumEnums: 3,
|
||||||
NumMessages: 33,
|
NumMessages: 34,
|
||||||
NumExtensions: 0,
|
NumExtensions: 0,
|
||||||
NumServices: 1,
|
NumServices: 1,
|
||||||
},
|
},
|
||||||
|
|||||||
Loading…
x
Reference in New Issue
Block a user