From f9827392bb2f4f5501dfd3fe90567a1fc2661805 Mon Sep 17 00:00:00 2001 From: Bingyi Sun Date: Mon, 29 Dec 2025 19:53:21 +0800 Subject: [PATCH] enhance: implement external collection update task with source change detection (#45905) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit issue: #45881 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. - Core invariant: at most one active UpdateExternalCollection task exists per collection — tasks are serialized by collectionID (collection-level locking) and any change to external_source or external_spec aborts superseded tasks and causes a new task creation (externalCollectionManager + external_collection_task_meta collection-based locks enforce this). - What was simplified/removed: per-task fine-grained locking and concurrent multi-task acceptance per collection were replaced by collection-level synchronization (external_collection_task_meta.go) and a single persistent task lifecycle in DataCoord/Index task code; redundant double-concurrent update paths were removed by checking existing task presence in AddTask/LoadOrStore and aborting/overwriting via Drop/Cancel flows. - Why this does NOT cause data loss or regress behavior: task state transitions and commit are validated against the current external source/spec before applying changes — UpdateStateWithMeta and SetJobInfo verify task metadata and persist via catalog only under matching collection-state; DataNode externalCollectionManager persists task results to in-memory manager and exposes Query/Drop flows (services.go) without modifying existing segment data unless a task successfully finishes and SetJobInfo atomically updates segments via meta/catalog calls, preventing superseded tasks from committing stale results. - New capability added: end-to-end external collection update workflow — DataCoord Index task + Cluster RPC helpers + DataNode external task runner and ExternalCollectionManager enable creating, querying, cancelling, and applying external collection updates (fragment-to-segment balancing, kept/updated segment handling, allocator integration); accompanying unit tests cover success, failure, cancellation, allocator errors, and balancing logic. --------- Signed-off-by: sunby --- .../external_collection_inspector.go | 4 +- .../external_collection_task_meta.go | 64 +- internal/datacoord/session/cluster.go | 75 + internal/datacoord/session/mock_cluster.go | 153 ++ .../task_update_external_collection.go | 270 ++- .../task_update_external_collection_test.go | 359 +++ internal/datanode/data_node.go | 8 + internal/datanode/external/manager.go | 272 +++ internal/datanode/external/manager_test.go | 439 ++++ internal/datanode/external/task_update.go | 456 ++++ .../datanode/external/task_update_test.go | 536 +++++ internal/datanode/services.go | 122 + pkg/proto/data_coord.proto | 28 + pkg/proto/datapb/data_coord.pb.go | 2136 ++++++++++------- pkg/taskcommon/type.go | 16 +- 15 files changed, 4010 insertions(+), 928 deletions(-) create mode 100644 internal/datacoord/task_update_external_collection_test.go create mode 100644 internal/datanode/external/manager.go create mode 100644 internal/datanode/external/manager_test.go create mode 100644 internal/datanode/external/task_update.go create mode 100644 internal/datanode/external/task_update_test.go diff --git a/internal/datacoord/external_collection_inspector.go b/internal/datacoord/external_collection_inspector.go index c2ba6569a5..23d8b4b3b7 100644 --- a/internal/datacoord/external_collection_inspector.go +++ b/internal/datacoord/external_collection_inspector.go @@ -88,7 +88,7 @@ func (i *externalCollectionInspector) reloadFromMeta() { continue } // Enqueue active tasks for processing - updateTask := newUpdateExternalCollectionTask(t, i.mt) + updateTask := newUpdateExternalCollectionTask(t, i.mt, i.allocator) i.scheduler.Enqueue(updateTask) } } @@ -253,7 +253,7 @@ func (i *externalCollectionInspector) SubmitUpdateTask(collectionID int64) error } // Create and enqueue task - updateTask := newUpdateExternalCollectionTask(t, i.mt) + updateTask := newUpdateExternalCollectionTask(t, i.mt, i.allocator) i.scheduler.Enqueue(updateTask) log.Info("external collection update task submitted", diff --git a/internal/datacoord/external_collection_task_meta.go b/internal/datacoord/external_collection_task_meta.go index e6d932521e..c257a2c903 100644 --- a/internal/datacoord/external_collection_task_meta.go +++ b/internal/datacoord/external_collection_task_meta.go @@ -76,15 +76,19 @@ func (ectm *externalCollectionTaskMeta) reloadFromKV() error { } func (ectm *externalCollectionTaskMeta) AddTask(t *indexpb.UpdateExternalCollectionTask) error { - ectm.keyLock.Lock(t.GetTaskID()) - defer ectm.keyLock.Unlock(t.GetTaskID()) + // Lock on collectionID to prevent concurrent tasks for the same collection + ectm.keyLock.Lock(t.GetCollectionID()) + defer ectm.keyLock.Unlock(t.GetCollectionID()) 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 { + // Check if a task already exists for this collection + if existingTask, ok := ectm.collectionID2Tasks.Get(t.GetCollectionID()); ok { log.Warn("update external collection task already exists for collection", + zap.Int64("existingTaskID", existingTask.GetTaskID()), + zap.Int64("newTaskID", t.GetTaskID()), zap.Int64("collectionID", t.GetCollectionID())) return merr.WrapErrTaskDuplicate(strconv.FormatInt(t.GetCollectionID(), 10)) } @@ -107,16 +111,28 @@ func (ectm *externalCollectionTaskMeta) AddTask(t *indexpb.UpdateExternalCollect } 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)) - + // First get the task to find its collectionID t, ok := ectm.tasks.Get(taskID) if !ok { - log.Info("remove update external collection task success, task already not exist", zap.Int64("taskID", taskID)) + log.Ctx(ctx).Info("remove update external collection task success, task already not exist", zap.Int64("taskID", taskID)) return nil } + + // Lock on collectionID to serialize with AddTask operations + ectm.keyLock.Lock(t.GetCollectionID()) + defer ectm.keyLock.Unlock(t.GetCollectionID()) + + log.Ctx(ctx).Info("drop update external collection task by taskID", + zap.Int64("taskID", taskID), + zap.Int64("collectionID", t.GetCollectionID())) + + // Double-check task still exists after acquiring lock + t, ok = ectm.tasks.Get(taskID) + if !ok { + log.Ctx(ctx).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), @@ -128,19 +144,28 @@ func (ectm *externalCollectionTaskMeta) DropTask(ctx context.Context, taskID int ectm.tasks.Remove(taskID) ectm.collectionID2Tasks.Remove(t.GetCollectionID()) - log.Info("remove update external collection task success", zap.Int64("taskID", taskID)) + log.Info("remove update external collection task success", + zap.Int64("taskID", taskID), + zap.Int64("collectionID", t.GetCollectionID())) 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) } + // Lock on collectionID for consistency with Add/Drop operations + ectm.keyLock.Lock(t.GetCollectionID()) + defer ectm.keyLock.Unlock(t.GetCollectionID()) + + // Double-check task still exists after acquiring lock + 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 @@ -162,14 +187,21 @@ func (ectm *externalCollectionTaskMeta) UpdateVersion(taskID, nodeID int64) erro } 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) } + // Lock on collectionID for consistency with Add/Drop operations + ectm.keyLock.Lock(t.GetCollectionID()) + defer ectm.keyLock.Unlock(t.GetCollectionID()) + + // Double-check task still exists after acquiring lock + 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 diff --git a/internal/datacoord/session/cluster.go b/internal/datacoord/session/cluster.go index 906f4adaa5..c0ba0312be 100644 --- a/internal/datacoord/session/cluster.go +++ b/internal/datacoord/session/cluster.go @@ -83,6 +83,13 @@ type Cluster interface { QueryAnalyze(nodeID int64, in *workerpb.QueryJobsRequest) (*workerpb.AnalyzeResults, error) // DropAnalyze drops an analysis task DropAnalyze(nodeID int64, taskID int64) error + + // CreateExternalCollectionTask creates and executes an external collection task + CreateExternalCollectionTask(nodeID int64, req *datapb.UpdateExternalCollectionRequest) error + // QueryExternalCollectionTask queries the status of an external collection task + QueryExternalCollectionTask(nodeID int64, taskID int64) (*datapb.UpdateExternalCollectionResponse, error) + // DropExternalCollectionTask drops an external collection task + DropExternalCollectionTask(nodeID int64, taskID int64) error } var _ Cluster = (*cluster)(nil) @@ -612,3 +619,71 @@ func (c *cluster) DropAnalyze(nodeID int64, taskID int64) error { properties.AppendType(taskcommon.Analyze) return c.dropTask(nodeID, properties) } + +func (c *cluster) CreateExternalCollectionTask(nodeID int64, req *datapb.UpdateExternalCollectionRequest) error { + timeout := paramtable.Get().DataCoordCfg.RequestTimeoutSeconds.GetAsDuration(time.Second) + ctx, cancel := context.WithTimeout(context.Background(), timeout) + defer cancel() + + cli, err := c.nm.GetClient(nodeID) + if err != nil { + log.Ctx(ctx).Warn("failed to get client", zap.Error(err)) + return err + } + + properties := taskcommon.NewProperties(nil) + properties.AppendClusterID(paramtable.Get().CommonCfg.ClusterPrefix.GetValue()) + properties.AppendTaskID(req.GetTaskID()) + properties.AppendType(taskcommon.ExternalCollection) + + payload, err := proto.Marshal(req) + if err != nil { + log.Ctx(ctx).Warn("marshal request failed", zap.Error(err)) + return err + } + + // Submit task to worker - task will execute asynchronously in worker's goroutine pool + status, err := cli.CreateTask(ctx, &workerpb.CreateTaskRequest{ + Payload: payload, + Properties: properties, + }) + if err != nil { + log.Ctx(ctx).Warn("create external collection task failed", zap.Error(err)) + return err + } + + if err := merr.Error(status); err != nil { + log.Ctx(ctx).Warn("create external collection task returned error", zap.Error(err)) + return err + } + + return nil +} + +func (c *cluster) QueryExternalCollectionTask(nodeID int64, taskID int64) (*datapb.UpdateExternalCollectionResponse, error) { + properties := taskcommon.NewProperties(nil) + properties.AppendClusterID(paramtable.Get().CommonCfg.ClusterPrefix.GetValue()) + properties.AppendTaskID(taskID) + properties.AppendType(taskcommon.ExternalCollection) + + resp, err := c.queryTask(nodeID, properties) + if err != nil { + return nil, err + } + + // Unmarshal the response payload + result := &datapb.UpdateExternalCollectionResponse{} + if err := proto.Unmarshal(resp.GetPayload(), result); err != nil { + return nil, err + } + + return result, nil +} + +func (c *cluster) DropExternalCollectionTask(nodeID int64, taskID int64) error { + properties := taskcommon.NewProperties(nil) + properties.AppendClusterID(paramtable.Get().CommonCfg.ClusterPrefix.GetValue()) + properties.AppendTaskID(taskID) + properties.AppendType(taskcommon.ExternalCollection) + return c.dropTask(nodeID, properties) +} diff --git a/internal/datacoord/session/mock_cluster.go b/internal/datacoord/session/mock_cluster.go index 123a3d08b1..63cfff3dc9 100644 --- a/internal/datacoord/session/mock_cluster.go +++ b/internal/datacoord/session/mock_cluster.go @@ -116,6 +116,53 @@ func (_c *MockCluster_CreateCompaction_Call) RunAndReturn(run func(int64, *datap return _c } +// CreateExternalCollectionTask provides a mock function with given fields: nodeID, req +func (_m *MockCluster) CreateExternalCollectionTask(nodeID int64, req *datapb.UpdateExternalCollectionRequest) error { + ret := _m.Called(nodeID, req) + + if len(ret) == 0 { + panic("no return value specified for CreateExternalCollectionTask") + } + + var r0 error + if rf, ok := ret.Get(0).(func(int64, *datapb.UpdateExternalCollectionRequest) error); ok { + r0 = rf(nodeID, req) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockCluster_CreateExternalCollectionTask_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CreateExternalCollectionTask' +type MockCluster_CreateExternalCollectionTask_Call struct { + *mock.Call +} + +// CreateExternalCollectionTask is a helper method to define mock.On call +// - nodeID int64 +// - req *datapb.UpdateExternalCollectionRequest +func (_e *MockCluster_Expecter) CreateExternalCollectionTask(nodeID interface{}, req interface{}) *MockCluster_CreateExternalCollectionTask_Call { + return &MockCluster_CreateExternalCollectionTask_Call{Call: _e.mock.On("CreateExternalCollectionTask", nodeID, req)} +} + +func (_c *MockCluster_CreateExternalCollectionTask_Call) Run(run func(nodeID int64, req *datapb.UpdateExternalCollectionRequest)) *MockCluster_CreateExternalCollectionTask_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(int64), args[1].(*datapb.UpdateExternalCollectionRequest)) + }) + return _c +} + +func (_c *MockCluster_CreateExternalCollectionTask_Call) Return(_a0 error) *MockCluster_CreateExternalCollectionTask_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockCluster_CreateExternalCollectionTask_Call) RunAndReturn(run func(int64, *datapb.UpdateExternalCollectionRequest) error) *MockCluster_CreateExternalCollectionTask_Call { + _c.Call.Return(run) + return _c +} + // CreateImport provides a mock function with given fields: nodeID, in, taskSlot func (_m *MockCluster) CreateImport(nodeID int64, in *datapb.ImportRequest, taskSlot int64) error { ret := _m.Called(nodeID, in, taskSlot) @@ -400,6 +447,53 @@ func (_c *MockCluster_DropCompaction_Call) RunAndReturn(run func(int64, int64) e return _c } +// DropExternalCollectionTask provides a mock function with given fields: nodeID, taskID +func (_m *MockCluster) DropExternalCollectionTask(nodeID int64, taskID int64) error { + ret := _m.Called(nodeID, taskID) + + if len(ret) == 0 { + panic("no return value specified for DropExternalCollectionTask") + } + + var r0 error + if rf, ok := ret.Get(0).(func(int64, int64) error); ok { + r0 = rf(nodeID, taskID) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockCluster_DropExternalCollectionTask_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'DropExternalCollectionTask' +type MockCluster_DropExternalCollectionTask_Call struct { + *mock.Call +} + +// DropExternalCollectionTask is a helper method to define mock.On call +// - nodeID int64 +// - taskID int64 +func (_e *MockCluster_Expecter) DropExternalCollectionTask(nodeID interface{}, taskID interface{}) *MockCluster_DropExternalCollectionTask_Call { + return &MockCluster_DropExternalCollectionTask_Call{Call: _e.mock.On("DropExternalCollectionTask", nodeID, taskID)} +} + +func (_c *MockCluster_DropExternalCollectionTask_Call) Run(run func(nodeID int64, taskID int64)) *MockCluster_DropExternalCollectionTask_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(int64), args[1].(int64)) + }) + return _c +} + +func (_c *MockCluster_DropExternalCollectionTask_Call) Return(_a0 error) *MockCluster_DropExternalCollectionTask_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockCluster_DropExternalCollectionTask_Call) RunAndReturn(run func(int64, int64) error) *MockCluster_DropExternalCollectionTask_Call { + _c.Call.Return(run) + return _c +} + // DropImport provides a mock function with given fields: nodeID, taskID func (_m *MockCluster) DropImport(nodeID int64, taskID int64) error { ret := _m.Called(nodeID, taskID) @@ -659,6 +753,65 @@ func (_c *MockCluster_QueryCompaction_Call) RunAndReturn(run func(int64, *datapb return _c } +// QueryExternalCollectionTask provides a mock function with given fields: nodeID, taskID +func (_m *MockCluster) QueryExternalCollectionTask(nodeID int64, taskID int64) (*datapb.UpdateExternalCollectionResponse, error) { + ret := _m.Called(nodeID, taskID) + + if len(ret) == 0 { + panic("no return value specified for QueryExternalCollectionTask") + } + + var r0 *datapb.UpdateExternalCollectionResponse + var r1 error + if rf, ok := ret.Get(0).(func(int64, int64) (*datapb.UpdateExternalCollectionResponse, error)); ok { + return rf(nodeID, taskID) + } + if rf, ok := ret.Get(0).(func(int64, int64) *datapb.UpdateExternalCollectionResponse); ok { + r0 = rf(nodeID, taskID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*datapb.UpdateExternalCollectionResponse) + } + } + + if rf, ok := ret.Get(1).(func(int64, int64) error); ok { + r1 = rf(nodeID, taskID) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockCluster_QueryExternalCollectionTask_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'QueryExternalCollectionTask' +type MockCluster_QueryExternalCollectionTask_Call struct { + *mock.Call +} + +// QueryExternalCollectionTask is a helper method to define mock.On call +// - nodeID int64 +// - taskID int64 +func (_e *MockCluster_Expecter) QueryExternalCollectionTask(nodeID interface{}, taskID interface{}) *MockCluster_QueryExternalCollectionTask_Call { + return &MockCluster_QueryExternalCollectionTask_Call{Call: _e.mock.On("QueryExternalCollectionTask", nodeID, taskID)} +} + +func (_c *MockCluster_QueryExternalCollectionTask_Call) Run(run func(nodeID int64, taskID int64)) *MockCluster_QueryExternalCollectionTask_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(int64), args[1].(int64)) + }) + return _c +} + +func (_c *MockCluster_QueryExternalCollectionTask_Call) Return(_a0 *datapb.UpdateExternalCollectionResponse, _a1 error) *MockCluster_QueryExternalCollectionTask_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockCluster_QueryExternalCollectionTask_Call) RunAndReturn(run func(int64, int64) (*datapb.UpdateExternalCollectionResponse, error)) *MockCluster_QueryExternalCollectionTask_Call { + _c.Call.Return(run) + return _c +} + // QueryImport provides a mock function with given fields: nodeID, in func (_m *MockCluster) QueryImport(nodeID int64, in *datapb.QueryImportRequest) (*datapb.QueryImportResponse, error) { ret := _m.Called(nodeID, in) diff --git a/internal/datacoord/task_update_external_collection.go b/internal/datacoord/task_update_external_collection.go index 843402ea09..67c2e67061 100644 --- a/internal/datacoord/task_update_external_collection.go +++ b/internal/datacoord/task_update_external_collection.go @@ -21,11 +21,15 @@ import ( "fmt" "time" + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "go.uber.org/zap" + "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/datacoord/session" globalTask "github.com/milvus-io/milvus/internal/datacoord/task" + "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/pkg/v2/log" + "github.com/milvus-io/milvus/pkg/v2/proto/datapb" "github.com/milvus-io/milvus/pkg/v2/proto/indexpb" "github.com/milvus-io/milvus/pkg/v2/taskcommon" ) @@ -35,16 +39,18 @@ type updateExternalCollectionTask struct { times *taskcommon.Times - meta *meta + meta *meta + allocator allocator.Allocator } var _ globalTask.Task = (*updateExternalCollectionTask)(nil) -func newUpdateExternalCollectionTask(t *indexpb.UpdateExternalCollectionTask, mt *meta) *updateExternalCollectionTask { +func newUpdateExternalCollectionTask(t *indexpb.UpdateExternalCollectionTask, mt *meta, alloc allocator.Allocator) *updateExternalCollectionTask { return &updateExternalCollectionTask{ UpdateExternalCollectionTask: t, times: taskcommon.NewTimes(), meta: mt, + allocator: alloc, } } @@ -118,32 +124,264 @@ func (t *updateExternalCollectionTask) SetState(state indexpb.JobState, failReas t.FailReason = failReason } -func (t *updateExternalCollectionTask) CreateTaskOnWorker(nodeID int64, cluster session.Cluster) { - ctx := context.Background() +func (t *updateExternalCollectionTask) UpdateStateWithMeta(state indexpb.JobState, failReason string) error { + if err := t.meta.externalCollectionTaskMeta.UpdateTaskState(t.GetTaskID(), state, failReason); err != nil { + log.Warn("update external collection task state failed", + zap.Int64("taskID", t.GetTaskID()), + zap.String("state", state.String()), + zap.String("failReason", failReason), + zap.Error(err)) + return err + } + t.SetState(state, failReason) + return nil +} + +// SetJobInfo processes the task response and updates segment information atomically +func (t *updateExternalCollectionTask) SetJobInfo(ctx context.Context, resp *datapb.UpdateExternalCollectionResponse) error { 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 + keptSegmentIDs := resp.GetKeptSegments() + updatedSegments := resp.GetUpdatedSegments() - log.Info("updating external collection metadata") + log.Info("processing external collection update response", + zap.Int("keptSegments", len(keptSegmentIDs)), + zap.Int("updatedSegments", len(updatedSegments))) - // TODO: Implement actual update logic - // For now, just mark as finished - t.SetState(indexpb.JobState_JobStateFinished, "") + // Build kept segments map for fast lookup + keptSegmentMap := make(map[int64]bool) + for _, segID := range keptSegmentIDs { + keptSegmentMap[segID] = true + } - log.Info("external collection metadata updated successfully") + // Allocate new IDs and update updatedSegments directly + for _, seg := range updatedSegments { + newSegmentID, err := t.allocator.AllocID(ctx) + if err != nil { + log.Warn("failed to allocate segment ID", zap.Error(err)) + return err + } + log.Info("allocated new segment ID", + zap.Int64("oldID", seg.GetID()), + zap.Int64("newID", newSegmentID)) + seg.ID = newSegmentID + seg.State = commonpb.SegmentState_Flushed + } + + // Build update operators + var operators []UpdateOperator + + // Operator 1: Drop segments not in kept list + dropOperator := func(modPack *updateSegmentPack) bool { + currentSegments := modPack.meta.segments.GetSegments() + for _, seg := range currentSegments { + // Skip segments not in this collection + if seg.GetCollectionID() != t.GetCollectionID() { + continue + } + + // Skip segments that are already dropped + if seg.GetState() == commonpb.SegmentState_Dropped { + continue + } + + // Drop segment if not in kept list + if !keptSegmentMap[seg.GetID()] { + segment := modPack.Get(seg.GetID()) + if segment != nil { + updateSegStateAndPrepareMetrics(segment, commonpb.SegmentState_Dropped, modPack.metricMutation) + segment.DroppedAt = uint64(time.Now().UnixNano()) + modPack.segments[seg.GetID()] = segment + log.Info("marking segment as dropped", + zap.Int64("segmentID", seg.GetID()), + zap.Int64("numRows", seg.GetNumOfRows())) + } + } + } + return true + } + operators = append(operators, dropOperator) + + // Operator 2: Add new segments + for _, seg := range updatedSegments { + newSeg := seg // capture for closure + addOperator := func(modPack *updateSegmentPack) bool { + segInfo := NewSegmentInfo(newSeg) + modPack.segments[newSeg.GetID()] = segInfo + + // Add binlogs increment + modPack.increments[newSeg.GetID()] = metastore.BinlogsIncrement{ + Segment: newSeg, + } + + // Update metrics + modPack.metricMutation.addNewSeg( + commonpb.SegmentState_Flushed, + newSeg.GetLevel(), + newSeg.GetIsSorted(), + newSeg.GetNumOfRows(), + ) + + log.Info("adding new segment", + zap.Int64("segmentID", newSeg.GetID()), + zap.Int64("numRows", newSeg.GetNumOfRows())) + return true + } + operators = append(operators, addOperator) + } + + // Execute all operators atomically + if err := t.meta.UpdateSegmentsInfo(ctx, operators...); err != nil { + log.Warn("failed to update segments atomically", zap.Error(err)) + return err + } + + log.Info("external collection segments updated successfully", + zap.Int("updatedSegments", len(updatedSegments)), + zap.Int("keptSegments", len(keptSegmentIDs))) + + return nil +} + +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()), + zap.Int64("nodeID", nodeID), + ) + + var err error + defer func() { + if err != nil { + log.Warn("failed to create external collection update task on worker", zap.Error(err)) + t.UpdateStateWithMeta(indexpb.JobState_JobStateFailed, err.Error()) + } + }() + + log.Info("creating external collection update task on worker") + + // Set node ID for this task + t.NodeID = nodeID + + // Get current segments for the collection + segments := t.meta.SelectSegments(ctx, CollectionFilter(t.GetCollectionID())) + + currentSegments := make([]*datapb.SegmentInfo, 0, len(segments)) + for _, seg := range segments { + currentSegments = append(currentSegments, seg.SegmentInfo) + } + + log.Info("collected current segments", zap.Int("segmentCount", len(currentSegments))) + + // Build request + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: t.GetCollectionID(), + TaskID: t.GetTaskID(), + CurrentSegments: currentSegments, + ExternalSource: t.GetExternalSource(), + ExternalSpec: t.GetExternalSpec(), + } + + // Submit task to worker via unified task system + // Task will execute asynchronously in worker's goroutine pool + err = cluster.CreateExternalCollectionTask(nodeID, req) + if err != nil { + log.Warn("failed to create external collection task on worker", zap.Error(err)) + return + } + + // Mark task as in progress - QueryTaskOnWorker will check completion + if err = t.UpdateStateWithMeta(indexpb.JobState_JobStateInProgress, ""); err != nil { + log.Warn("failed to update task state to InProgress", zap.Error(err)) + return + } + + log.Info("external collection update task submitted successfully") } func (t *updateExternalCollectionTask) QueryTaskOnWorker(cluster session.Cluster) { - // External collection tasks finish immediately, so query is a no-op + ctx := context.Background() + log := log.Ctx(ctx).With( + zap.Int64("taskID", t.GetTaskID()), + zap.Int64("collectionID", t.GetCollectionID()), + zap.Int64("nodeID", t.GetNodeID()), + ) + + // Query task status from worker + resp, err := cluster.QueryExternalCollectionTask(t.GetNodeID(), t.GetTaskID()) + if err != nil { + log.Warn("query external collection task result failed", zap.Error(err)) + // If query fails, mark task as failed + t.UpdateStateWithMeta(indexpb.JobState_JobStateFailed, fmt.Sprintf("query task failed: %v", err)) + return + } + + state := resp.GetState() + failReason := resp.GetFailReason() + + log.Info("queried external collection task status", + zap.String("state", state.String()), + zap.String("failReason", failReason)) + + // Handle different task states + switch state { + case indexpb.JobState_JobStateFinished: + // Process the response and update segment info + if err := t.SetJobInfo(ctx, resp); err != nil { + log.Warn("failed to process job info", zap.Error(err)) + t.UpdateStateWithMeta(indexpb.JobState_JobStateFailed, fmt.Sprintf("failed to process job info: %v", err)) + return + } + + // Task completed successfully + if err := t.UpdateStateWithMeta(state, ""); err != nil { + log.Warn("failed to update task state to Finished", zap.Error(err)) + return + } + log.Info("external collection task completed successfully") + + case indexpb.JobState_JobStateFailed: + // Task failed + if err := t.UpdateStateWithMeta(state, failReason); err != nil { + log.Warn("failed to update task state to Failed", zap.Error(err)) + return + } + log.Warn("external collection task failed", zap.String("reason", failReason)) + + case indexpb.JobState_JobStateInProgress: + // Task still in progress, no action needed + log.Info("external collection task still in progress") + + case indexpb.JobState_JobStateNone, indexpb.JobState_JobStateRetry: + // Task not found or needs retry - mark as failed + log.Warn("external collection task in unexpected state, marking as failed", + zap.String("state", state.String())) + t.UpdateStateWithMeta(indexpb.JobState_JobStateFailed, fmt.Sprintf("task in unexpected state: %s", state.String())) + + default: + log.Warn("external collection task in unknown state", + zap.String("state", state.String())) + } } func (t *updateExternalCollectionTask) DropTaskOnWorker(cluster session.Cluster) { - // External collection tasks don't run on workers, so drop is a no-op + ctx := context.Background() + log := log.Ctx(ctx).With( + zap.Int64("taskID", t.GetTaskID()), + zap.Int64("collectionID", t.GetCollectionID()), + zap.Int64("nodeID", t.GetNodeID()), + ) + + // Drop task on worker to cancel execution and clean up resources + err := cluster.DropExternalCollectionTask(t.GetNodeID(), t.GetTaskID()) + if err != nil { + log.Warn("failed to drop external collection task on worker", zap.Error(err)) + return + } + + log.Info("external collection task dropped successfully") } diff --git a/internal/datacoord/task_update_external_collection_test.go b/internal/datacoord/task_update_external_collection_test.go new file mode 100644 index 0000000000..703b14a6a5 --- /dev/null +++ b/internal/datacoord/task_update_external_collection_test.go @@ -0,0 +1,359 @@ +// 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" + "testing" + + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/suite" + + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus/internal/datacoord/allocator" + "github.com/milvus-io/milvus/internal/datacoord/broker" + "github.com/milvus-io/milvus/internal/datacoord/session" + kvcatalog "github.com/milvus-io/milvus/internal/metastore/kv/datacoord" + "github.com/milvus-io/milvus/pkg/v2/proto/datapb" + "github.com/milvus-io/milvus/pkg/v2/proto/indexpb" + "github.com/milvus-io/milvus/pkg/v2/proto/rootcoordpb" + "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/typeutil" +) + +type updateExternalCollectionTaskSuite struct { + suite.Suite + mt *meta + + collID int64 + taskID int64 + nodeID int64 + + externalSource string + externalSpec string +} + +func Test_updateExternalCollectionTaskSuite(t *testing.T) { + suite.Run(t, new(updateExternalCollectionTaskSuite)) +} + +func (s *updateExternalCollectionTaskSuite) SetupSuite() { + s.collID = 100 + s.taskID = 200 + s.nodeID = 1 + s.externalSource = "s3" + s.externalSpec = "spec" +} + +func (s *updateExternalCollectionTaskSuite) SetupTest() { + catalog := kvcatalog.NewCatalog(NewMetaMemoryKV(), "", "") + mockBroker := broker.NewMockBroker(s.T()) + mockBroker.EXPECT().ShowCollectionIDs(mock.Anything).Return(&rootcoordpb.ShowCollectionIDsResponse{ + Status: merr.Success(), + DbCollections: []*rootcoordpb.DBCollections{ + { + DbName: "default", + CollectionIDs: []int64{s.collID}, + }, + }, + }, nil) + + mt, err := newMeta(context.Background(), catalog, nil, mockBroker) + s.Require().NoError(err) + s.mt = mt + // ensure each test starts from a clean segment/task state + s.mt.segments = NewSegmentsInfo() + s.mt.externalCollectionTaskMeta = &externalCollectionTaskMeta{ + ctx: context.Background(), + catalog: catalog, + keyLock: lock.NewKeyLock[UniqueID](), + tasks: typeutil.NewConcurrentMap[UniqueID, *indexpb.UpdateExternalCollectionTask](), + collectionID2Tasks: typeutil.NewConcurrentMap[UniqueID, *indexpb.UpdateExternalCollectionTask](), + } + + collection := &collectionInfo{ + ID: s.collID, + Schema: newTestSchema(), + } + collection.Schema.ExternalSource = s.externalSource + collection.Schema.ExternalSpec = s.externalSpec + s.mt.AddCollection(collection) +} + +func (s *updateExternalCollectionTaskSuite) TestSetJobInfo_KeepAndAddSegments() { + // Setup: Create initial segments + seg1 := &SegmentInfo{ + SegmentInfo: &datapb.SegmentInfo{ + ID: 1, + CollectionID: s.collID, + State: commonpb.SegmentState_Flushed, + NumOfRows: 1000, + }, + } + seg2 := &SegmentInfo{ + SegmentInfo: &datapb.SegmentInfo{ + ID: 2, + CollectionID: s.collID, + State: commonpb.SegmentState_Flushed, + NumOfRows: 2000, + }, + } + s.mt.segments.SetSegment(1, seg1) + s.mt.segments.SetSegment(2, seg2) + + // Create mock allocator + mockAlloc := allocator.NewMockAllocator(s.T()) + mockAlloc.EXPECT().AllocID(mock.Anything).Return(int64(100), nil).Once() + mockAlloc.EXPECT().AllocID(mock.Anything).Return(int64(101), nil).Once() + + // Create task + task := newUpdateExternalCollectionTask(&indexpb.UpdateExternalCollectionTask{ + TaskID: s.taskID, + CollectionID: s.collID, + ExternalSource: s.externalSource, + ExternalSpec: s.externalSpec, + State: indexpb.JobState_JobStateInit, + }, s.mt, mockAlloc) + + // Create response: keep segment 1, drop segment 2, add 2 new segments + resp := &datapb.UpdateExternalCollectionResponse{ + KeptSegments: []int64{1}, + UpdatedSegments: []*datapb.SegmentInfo{ + { + ID: 10, // placeholder + CollectionID: s.collID, + State: commonpb.SegmentState_Flushed, + NumOfRows: 3000, + }, + { + ID: 20, // placeholder + CollectionID: s.collID, + State: commonpb.SegmentState_Flushed, + NumOfRows: 4000, + }, + }, + State: indexpb.JobState_JobStateFinished, + } + + // Execute + err := task.SetJobInfo(context.Background(), resp) + s.NoError(err) + + // Verify: segment 1 should still be there and flushed + seg1After := s.mt.segments.GetSegment(1) + s.NotNil(seg1After) + s.Equal(commonpb.SegmentState_Flushed, seg1After.GetState()) + + // Verify: segment 2 should be dropped + seg2After := s.mt.segments.GetSegment(2) + s.NotNil(seg2After) + s.Equal(commonpb.SegmentState_Dropped, seg2After.GetState()) + + // Verify: new segments should be added with allocated IDs + newSeg1 := s.mt.segments.GetSegment(100) + s.NotNil(newSeg1) + s.Equal(int64(3000), newSeg1.GetNumOfRows()) + s.Equal(commonpb.SegmentState_Flushed, newSeg1.GetState()) + + newSeg2 := s.mt.segments.GetSegment(101) + s.NotNil(newSeg2) + s.Equal(int64(4000), newSeg2.GetNumOfRows()) + s.Equal(commonpb.SegmentState_Flushed, newSeg2.GetState()) +} + +func (s *updateExternalCollectionTaskSuite) TestSetJobInfo_DropAllSegments() { + // Setup: Create initial segments + seg1 := &SegmentInfo{ + SegmentInfo: &datapb.SegmentInfo{ + ID: 1, + CollectionID: s.collID, + State: commonpb.SegmentState_Flushed, + NumOfRows: 1000, + }, + } + seg2 := &SegmentInfo{ + SegmentInfo: &datapb.SegmentInfo{ + ID: 2, + CollectionID: s.collID, + State: commonpb.SegmentState_Flushed, + NumOfRows: 2000, + }, + } + s.mt.segments.SetSegment(1, seg1) + s.mt.segments.SetSegment(2, seg2) + + // Create mock allocator + mockAlloc := allocator.NewMockAllocator(s.T()) + + // Create task + task := newUpdateExternalCollectionTask(&indexpb.UpdateExternalCollectionTask{ + TaskID: s.taskID, + CollectionID: s.collID, + ExternalSource: s.externalSource, + ExternalSpec: s.externalSpec, + State: indexpb.JobState_JobStateInit, + }, s.mt, mockAlloc) + + // Create response: drop all segments, no new segments + resp := &datapb.UpdateExternalCollectionResponse{ + KeptSegments: []int64{}, + UpdatedSegments: []*datapb.SegmentInfo{}, + State: indexpb.JobState_JobStateFinished, + } + + // Execute + err := task.SetJobInfo(context.Background(), resp) + s.NoError(err) + + // Verify: all segments should be dropped + seg1After := s.mt.segments.GetSegment(1) + s.NotNil(seg1After) + s.Equal(commonpb.SegmentState_Dropped, seg1After.GetState()) + + seg2After := s.mt.segments.GetSegment(2) + s.NotNil(seg2After) + s.Equal(commonpb.SegmentState_Dropped, seg2After.GetState()) +} + +func (s *updateExternalCollectionTaskSuite) TestSetJobInfo_AllocatorError() { + // Create mock allocator that fails + mockAlloc := allocator.NewMockAllocator(s.T()) + mockAlloc.EXPECT().AllocID(mock.Anything).Return(int64(0), errors.New("alloc failed")) + + // Create task + task := newUpdateExternalCollectionTask(&indexpb.UpdateExternalCollectionTask{ + TaskID: s.taskID, + CollectionID: s.collID, + ExternalSource: s.externalSource, + ExternalSpec: s.externalSpec, + State: indexpb.JobState_JobStateInit, + }, s.mt, mockAlloc) + + // Create response with new segments + resp := &datapb.UpdateExternalCollectionResponse{ + KeptSegments: []int64{}, + UpdatedSegments: []*datapb.SegmentInfo{ + { + ID: 10, + CollectionID: s.collID, + NumOfRows: 1000, + }, + }, + State: indexpb.JobState_JobStateFinished, + } + + // Execute + err := task.SetJobInfo(context.Background(), resp) + s.Error(err) + s.Contains(err.Error(), "alloc failed") +} + +func (s *updateExternalCollectionTaskSuite) TestQueryTaskOnWorker_Success() { + // Create mock allocator + mockAlloc := allocator.NewMockAllocator(s.T()) + mockAlloc.EXPECT().AllocID(mock.Anything).Return(int64(100), nil).Maybe() + + // Create task + task := newUpdateExternalCollectionTask(&indexpb.UpdateExternalCollectionTask{ + TaskID: s.taskID, + CollectionID: s.collID, + State: indexpb.JobState_JobStateInProgress, + NodeID: s.nodeID, + ExternalSource: s.externalSource, + ExternalSpec: s.externalSpec, + }, s.mt, mockAlloc) + + // Add task to meta + s.mt.externalCollectionTaskMeta.tasks.Insert(s.taskID, task.UpdateExternalCollectionTask) + + // Create mock cluster + mockCluster := session.NewMockCluster(s.T()) + mockCluster.EXPECT().QueryExternalCollectionTask(s.nodeID, s.taskID).Return(&datapb.UpdateExternalCollectionResponse{ + State: indexpb.JobState_JobStateFinished, + KeptSegments: []int64{}, + UpdatedSegments: []*datapb.SegmentInfo{}, + }, nil) + + // Execute + task.QueryTaskOnWorker(mockCluster) + + // Verify task state is finished + s.Equal(indexpb.JobState_JobStateFinished, task.GetState()) +} + +func (s *updateExternalCollectionTaskSuite) TestQueryTaskOnWorker_Failed() { + // Create mock allocator + mockAlloc := allocator.NewMockAllocator(s.T()) + + // Create task + task := newUpdateExternalCollectionTask(&indexpb.UpdateExternalCollectionTask{ + TaskID: s.taskID, + CollectionID: s.collID, + State: indexpb.JobState_JobStateInProgress, + NodeID: s.nodeID, + ExternalSource: s.externalSource, + ExternalSpec: s.externalSpec, + }, s.mt, mockAlloc) + + // Add task to meta + s.mt.externalCollectionTaskMeta.tasks.Insert(s.taskID, task.UpdateExternalCollectionTask) + + // Create mock cluster that returns failed state + mockCluster := session.NewMockCluster(s.T()) + mockCluster.EXPECT().QueryExternalCollectionTask(s.nodeID, s.taskID).Return(&datapb.UpdateExternalCollectionResponse{ + State: indexpb.JobState_JobStateFailed, + FailReason: "task execution failed", + }, nil) + + // Execute + task.QueryTaskOnWorker(mockCluster) + + // Verify task state is failed + s.Equal(indexpb.JobState_JobStateFailed, task.GetState()) + s.Equal("task execution failed", task.GetFailReason()) +} + +func (s *updateExternalCollectionTaskSuite) TestQueryTaskOnWorker_QueryError() { + // Create mock allocator + mockAlloc := allocator.NewMockAllocator(s.T()) + + // Create task + task := newUpdateExternalCollectionTask(&indexpb.UpdateExternalCollectionTask{ + TaskID: s.taskID, + CollectionID: s.collID, + State: indexpb.JobState_JobStateInProgress, + NodeID: s.nodeID, + ExternalSource: s.externalSource, + ExternalSpec: s.externalSpec, + }, s.mt, mockAlloc) + + // Add task to meta + s.mt.externalCollectionTaskMeta.tasks.Insert(s.taskID, task.UpdateExternalCollectionTask) + + // Create mock cluster that returns error + mockCluster := session.NewMockCluster(s.T()) + mockCluster.EXPECT().QueryExternalCollectionTask(s.nodeID, s.taskID).Return(nil, errors.New("query failed")) + + // Execute + task.QueryTaskOnWorker(mockCluster) + + // Verify task state is failed + s.Equal(indexpb.JobState_JobStateFailed, task.GetState()) + s.Contains(task.GetFailReason(), "query task failed") +} diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index 09ee0b6a6d..7b46f5f68e 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -36,6 +36,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/compaction" "github.com/milvus-io/milvus/internal/datanode/compactor" + "github.com/milvus-io/milvus/internal/datanode/external" "github.com/milvus-io/milvus/internal/datanode/importv2" "github.com/milvus-io/milvus/internal/datanode/index" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" @@ -87,6 +88,8 @@ type DataNode struct { taskScheduler *index.TaskScheduler taskManager *index.TaskManager + externalCollectionManager *external.ExternalCollectionManager + compactionExecutor compactor.Executor etcdCli *clientv3.Client @@ -124,6 +127,7 @@ func NewDataNode(ctx context.Context) *DataNode { node.storageFactory = NewChunkMgrFactory() node.taskScheduler = sc node.taskManager = index.NewTaskManager(ctx2) + node.externalCollectionManager = external.NewExternalCollectionManager(ctx2, 8) node.UpdateStateCode(commonpb.StateCode_Abnormal) expr.Register("datanode", node) return node @@ -300,6 +304,10 @@ func (node *DataNode) Stop() error { node.importScheduler.Close() } + if node.externalCollectionManager != nil { + node.externalCollectionManager.Close() + } + // cleanup all running tasks node.taskManager.DeleteAllTasks() diff --git a/internal/datanode/external/manager.go b/internal/datanode/external/manager.go new file mode 100644 index 0000000000..e941c00bdc --- /dev/null +++ b/internal/datanode/external/manager.go @@ -0,0 +1,272 @@ +// 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 external + +import ( + "context" + "fmt" + "sync" + + "go.uber.org/zap" + "google.golang.org/protobuf/proto" + + "github.com/milvus-io/milvus/pkg/v2/log" + "github.com/milvus-io/milvus/pkg/v2/proto/datapb" + "github.com/milvus-io/milvus/pkg/v2/proto/indexpb" + "github.com/milvus-io/milvus/pkg/v2/util/conc" +) + +// TaskKey uniquely identifies an external collection task. +type TaskKey struct { + ClusterID string + TaskID int64 +} + +// TaskInfo stores the mutable state of an external collection task. +type TaskInfo struct { + Cancel context.CancelFunc + State indexpb.JobState + FailReason string + CollID int64 + KeptSegments []int64 + UpdatedSegments []*datapb.SegmentInfo +} + +// Clone creates a deep copy so callers can freely mutate the result. +func (t *TaskInfo) Clone() *TaskInfo { + return &TaskInfo{ + Cancel: t.Cancel, + State: t.State, + FailReason: t.FailReason, + CollID: t.CollID, + KeptSegments: cloneSegmentIDs(t.KeptSegments), + UpdatedSegments: cloneSegments(t.UpdatedSegments), + } +} + +func makeTaskKey(clusterID string, taskID int64) TaskKey { + return TaskKey{ + ClusterID: clusterID, + TaskID: taskID, + } +} + +func cloneSegmentIDs(src []int64) []int64 { + if len(src) == 0 { + return nil + } + dst := make([]int64, len(src)) + copy(dst, src) + return dst +} + +func extractSegmentIDs(segments []*datapb.SegmentInfo) []int64 { + if len(segments) == 0 { + return nil + } + result := make([]int64, 0, len(segments)) + for _, seg := range segments { + if seg == nil { + continue + } + result = append(result, seg.GetID()) + } + return result +} + +// cloneSegments returns deep copies of SegmentInfo slices. +func cloneSegments(src []*datapb.SegmentInfo) []*datapb.SegmentInfo { + if len(src) == 0 { + return nil + } + cloned := make([]*datapb.SegmentInfo, len(src)) + for i, seg := range src { + if seg == nil { + continue + } + cloned[i] = proto.Clone(seg).(*datapb.SegmentInfo) + } + return cloned +} + +// ExternalCollectionManager supervises the lifecycle of external collection tasks +// within a single datanode. +type ExternalCollectionManager struct { + ctx context.Context + mu sync.RWMutex + tasks map[TaskKey]*TaskInfo + pool *conc.Pool[any] + closeOnce sync.Once +} + +// NewExternalCollectionManager constructs a manager with the provided worker pool size. +func NewExternalCollectionManager(ctx context.Context, poolSize int) *ExternalCollectionManager { + return &ExternalCollectionManager{ + ctx: ctx, + tasks: make(map[TaskKey]*TaskInfo), + pool: conc.NewPool[any](poolSize), + } +} + +// Close releases all background resources. +func (m *ExternalCollectionManager) Close() { + m.closeOnce.Do(func() { + if m.pool != nil { + m.pool.Release() + } + log.Info("external collection manager closed") + }) +} + +// LoadOrStore adds a task entry if absent and returns the existing entry if present. +func (m *ExternalCollectionManager) LoadOrStore(clusterID string, taskID int64, info *TaskInfo) *TaskInfo { + m.mu.Lock() + defer m.mu.Unlock() + key := makeTaskKey(clusterID, taskID) + if oldInfo, ok := m.tasks[key]; ok { + return oldInfo + } + m.tasks[key] = info + return nil +} + +// Get returns a cloned snapshot of a task. +func (m *ExternalCollectionManager) Get(clusterID string, taskID int64) *TaskInfo { + m.mu.RLock() + defer m.mu.RUnlock() + key := makeTaskKey(clusterID, taskID) + if info, ok := m.tasks[key]; ok { + return info.Clone() + } + return nil +} + +// Delete removes the task entry and returns the previous value. +func (m *ExternalCollectionManager) Delete(clusterID string, taskID int64) *TaskInfo { + m.mu.Lock() + defer m.mu.Unlock() + key := makeTaskKey(clusterID, taskID) + if info, ok := m.tasks[key]; ok { + delete(m.tasks, key) + return info + } + return nil +} + +// UpdateState updates only the state/failReason fields. +func (m *ExternalCollectionManager) UpdateState(clusterID string, taskID int64, state indexpb.JobState, failReason string) { + m.mu.Lock() + defer m.mu.Unlock() + key := makeTaskKey(clusterID, taskID) + if info, ok := m.tasks[key]; ok { + info.State = state + info.FailReason = failReason + } +} + +// UpdateResult commits the latest state plus kept/updated segments atomically. +func (m *ExternalCollectionManager) UpdateResult(clusterID string, taskID int64, + state indexpb.JobState, + failReason string, + keptSegments []int64, + updatedSegments []*datapb.SegmentInfo, +) { + m.mu.Lock() + defer m.mu.Unlock() + key := makeTaskKey(clusterID, taskID) + if info, ok := m.tasks[key]; ok { + info.State = state + info.FailReason = failReason + info.KeptSegments = append([]int64(nil), keptSegments...) + info.UpdatedSegments = cloneSegments(updatedSegments) + } +} + +// CancelTask triggers the context cancellation for a task if it exists. +func (m *ExternalCollectionManager) CancelTask(clusterID string, taskID int64) bool { + key := makeTaskKey(clusterID, taskID) + m.mu.RLock() + info, ok := m.tasks[key] + var cancel context.CancelFunc + if ok { + cancel = info.Cancel + } + m.mu.RUnlock() + if cancel != nil { + cancel() + } + return ok +} + +// SubmitTask registers and runs a task asynchronously in the manager pool. +func (m *ExternalCollectionManager) SubmitTask( + clusterID string, + req *datapb.UpdateExternalCollectionRequest, + taskFunc func(context.Context) (*datapb.UpdateExternalCollectionResponse, error), +) error { + taskID := req.GetTaskID() + + taskCtx, cancel := context.WithCancel(m.ctx) + keptSegments := extractSegmentIDs(req.GetCurrentSegments()) + + info := &TaskInfo{ + Cancel: cancel, + State: indexpb.JobState_JobStateInProgress, + FailReason: "", + CollID: req.GetCollectionID(), + KeptSegments: keptSegments, + UpdatedSegments: nil, + } + + if oldInfo := m.LoadOrStore(clusterID, taskID, info); oldInfo != nil { + return fmt.Errorf("task already exists: taskID=%d", taskID) + } + + // Submit to pool + m.pool.Submit(func() (any, error) { + defer cancel() + log.Info("executing external collection task in pool", + zap.Int64("taskID", taskID), + zap.Int64("collectionID", req.GetCollectionID())) + + // Execute the task + resp, err := taskFunc(taskCtx) + if err != nil { + m.UpdateResult(clusterID, taskID, indexpb.JobState_JobStateFailed, err.Error(), info.KeptSegments, nil) + log.Warn("external collection task failed", + zap.Int64("taskID", taskID), + zap.Error(err)) + return nil, err + } + + state := resp.GetState() + if state == indexpb.JobState_JobStateNone { + state = indexpb.JobState_JobStateFinished + } + failReason := resp.GetFailReason() + kept := resp.GetKeptSegments() + if len(kept) == 0 { + kept = info.KeptSegments + } + m.UpdateResult(clusterID, taskID, state, failReason, kept, resp.GetUpdatedSegments()) + log.Info("external collection task completed", + zap.Int64("taskID", taskID)) + return nil, nil + }) + + return nil +} diff --git a/internal/datanode/external/manager_test.go b/internal/datanode/external/manager_test.go new file mode 100644 index 0000000000..9e3e894dde --- /dev/null +++ b/internal/datanode/external/manager_test.go @@ -0,0 +1,439 @@ +// 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 external + +import ( + "context" + "errors" + "sync/atomic" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/milvus-io/milvus/pkg/v2/proto/datapb" + "github.com/milvus-io/milvus/pkg/v2/proto/indexpb" +) + +func TestExternalCollectionManager_Basic(t *testing.T) { + ctx := context.Background() + manager := NewExternalCollectionManager(ctx, 4) + defer manager.Close() + + clusterID := "test-cluster" + taskID := int64(1) + collID := int64(100) + + // Test LoadOrStore - first time should succeed + info := &TaskInfo{ + Cancel: func() {}, + State: indexpb.JobState_JobStateInProgress, + FailReason: "", + CollID: collID, + } + oldInfo := manager.LoadOrStore(clusterID, taskID, info) + assert.Nil(t, oldInfo) + + // Test Get + retrievedInfo := manager.Get(clusterID, taskID) + assert.NotNil(t, retrievedInfo) + assert.Equal(t, indexpb.JobState_JobStateInProgress, retrievedInfo.State) + assert.Equal(t, collID, retrievedInfo.CollID) + + // Test LoadOrStore - second time should return existing + newInfo := &TaskInfo{ + Cancel: func() {}, + State: indexpb.JobState_JobStateFinished, + FailReason: "", + CollID: collID, + } + oldInfo = manager.LoadOrStore(clusterID, taskID, newInfo) + assert.NotNil(t, oldInfo) + assert.Equal(t, indexpb.JobState_JobStateInProgress, oldInfo.State) // should still be old state + + // Test UpdateState + manager.UpdateState(clusterID, taskID, indexpb.JobState_JobStateFinished, "") + retrievedInfo = manager.Get(clusterID, taskID) + assert.Equal(t, indexpb.JobState_JobStateFinished, retrievedInfo.State) + + // Test Delete + deletedInfo := manager.Delete(clusterID, taskID) + assert.NotNil(t, deletedInfo) + assert.Equal(t, indexpb.JobState_JobStateFinished, deletedInfo.State) + + // Verify task is deleted + retrievedInfo = manager.Get(clusterID, taskID) + assert.Nil(t, retrievedInfo) +} + +func TestExternalCollectionManager_SubmitTask_Success(t *testing.T) { + ctx := context.Background() + manager := NewExternalCollectionManager(ctx, 4) + defer manager.Close() + + clusterID := "test-cluster" + taskID := int64(2) + collID := int64(200) + + req := &datapb.UpdateExternalCollectionRequest{ + TaskID: taskID, + CollectionID: collID, + } + + // Track task execution + var executed atomic.Bool + taskFunc := func(ctx context.Context) (*datapb.UpdateExternalCollectionResponse, error) { + executed.Store(true) + return &datapb.UpdateExternalCollectionResponse{ + State: indexpb.JobState_JobStateFinished, + KeptSegments: []int64{1, 2}, + }, nil + } + + // Submit task + err := manager.SubmitTask(clusterID, req, taskFunc) + assert.NoError(t, err) + + require.Eventually(t, func() bool { + return executed.Load() + }, time.Second, 10*time.Millisecond) + + require.Eventually(t, func() bool { + info := manager.Get(clusterID, taskID) + return info != nil && info.State == indexpb.JobState_JobStateFinished + }, time.Second, 10*time.Millisecond) + + // Verify task was executed + assert.True(t, executed.Load()) + + // Task info should be retained until explicit drop + info := manager.Get(clusterID, taskID) + assert.NotNil(t, info) + assert.Equal(t, indexpb.JobState_JobStateFinished, info.State) + assert.Equal(t, []int64{1, 2}, info.KeptSegments) +} + +func TestExternalCollectionManager_SubmitTask_Failure(t *testing.T) { + ctx := context.Background() + manager := NewExternalCollectionManager(ctx, 4) + defer manager.Close() + + clusterID := "test-cluster" + taskID := int64(3) + collID := int64(300) + + req := &datapb.UpdateExternalCollectionRequest{ + TaskID: taskID, + CollectionID: collID, + } + + // Task function that fails + expectedError := errors.New("task execution failed") + taskFunc := func(ctx context.Context) (*datapb.UpdateExternalCollectionResponse, error) { + return nil, expectedError + } + + // Submit task + err := manager.SubmitTask(clusterID, req, taskFunc) + assert.NoError(t, err) // Submit should succeed + + require.Eventually(t, func() bool { + info := manager.Get(clusterID, taskID) + return info != nil && info.State == indexpb.JobState_JobStateFailed + }, time.Second, 10*time.Millisecond) + + // Task info should still be present with failure state + info := manager.Get(clusterID, taskID) + assert.NotNil(t, info) + assert.Equal(t, indexpb.JobState_JobStateFailed, info.State) + assert.Equal(t, expectedError.Error(), info.FailReason) +} + +func TestExternalCollectionManager_CancelTask(t *testing.T) { + ctx := context.Background() + manager := NewExternalCollectionManager(ctx, 4) + defer manager.Close() + + clusterID := "test-cluster" + taskID := int64(30) + collID := int64(3000) + + req := &datapb.UpdateExternalCollectionRequest{ + TaskID: taskID, + CollectionID: collID, + } + + cancelObserved := make(chan struct{}) + taskFunc := func(ctx context.Context) (*datapb.UpdateExternalCollectionResponse, error) { + select { + case <-ctx.Done(): + close(cancelObserved) + return nil, ctx.Err() + case <-time.After(time.Second): + return &datapb.UpdateExternalCollectionResponse{ + State: indexpb.JobState_JobStateFinished, + }, nil + } + } + + err := manager.SubmitTask(clusterID, req, taskFunc) + assert.NoError(t, err) + + require.Eventually(t, func() bool { + // ensure task has been registered + info := manager.Get(clusterID, taskID) + return info != nil + }, time.Second, 10*time.Millisecond) + + cancelled := manager.CancelTask(clusterID, taskID) + assert.True(t, cancelled) + + require.Eventually(t, func() bool { + select { + case <-cancelObserved: + return true + default: + return false + } + }, time.Second, 10*time.Millisecond) + + info := manager.Get(clusterID, taskID) + require.NotNil(t, info) + assert.Equal(t, indexpb.JobState_JobStateFailed, info.State) + assert.Contains(t, info.FailReason, "context canceled") +} + +func TestCloneSegmentIDs(t *testing.T) { + src := []int64{1, 2, 3} + dst := cloneSegmentIDs(src) + + assert.Equal(t, src, dst) + dst[0] = 42 + assert.NotEqual(t, src[0], dst[0], "modifying clone should not affect source") +} + +func TestExtractSegmentIDs(t *testing.T) { + assert.Nil(t, extractSegmentIDs(nil)) + + segments := []*datapb.SegmentInfo{ + nil, + {ID: 1}, + {ID: 2}, + } + assert.Equal(t, []int64{1, 2}, extractSegmentIDs(segments)) +} + +func TestCancelTaskMultipleTimes(t *testing.T) { + ctx := context.Background() + manager := NewExternalCollectionManager(ctx, 1) + defer manager.Close() + + var calls int32 + cancelFn := func() { + atomic.AddInt32(&calls, 1) + } + + clusterID := "cluster" + taskID := int64(999) + + manager.LoadOrStore(clusterID, taskID, &TaskInfo{ + Cancel: cancelFn, + }) + + require.True(t, manager.CancelTask(clusterID, taskID)) + require.True(t, manager.CancelTask(clusterID, taskID)) + assert.Equal(t, int32(2), calls) +} + +func TestExternalCollectionManager_SubmitTask_Duplicate(t *testing.T) { + ctx := context.Background() + manager := NewExternalCollectionManager(ctx, 4) + defer manager.Close() + + clusterID := "test-cluster" + taskID := int64(4) + collID := int64(400) + + req := &datapb.UpdateExternalCollectionRequest{ + TaskID: taskID, + CollectionID: collID, + } + + // Task function that blocks + blockChan := make(chan struct{}) + taskFunc := func(ctx context.Context) (*datapb.UpdateExternalCollectionResponse, error) { + <-blockChan + return &datapb.UpdateExternalCollectionResponse{ + State: indexpb.JobState_JobStateFinished, + }, nil + } + + // Submit first task + err := manager.SubmitTask(clusterID, req, taskFunc) + assert.NoError(t, err) + + // Verify task is in progress + info := manager.Get(clusterID, taskID) + assert.NotNil(t, info) + assert.Equal(t, indexpb.JobState_JobStateInProgress, info.State) + + // Try to submit duplicate task + err = manager.SubmitTask(clusterID, req, taskFunc) + assert.Error(t, err) + assert.Contains(t, err.Error(), "task already exists") + + // Unblock the task + close(blockChan) + + require.Eventually(t, func() bool { + info := manager.Get(clusterID, taskID) + return info != nil && info.State == indexpb.JobState_JobStateFinished + }, time.Second, 10*time.Millisecond) +} + +func TestExternalCollectionManager_MultipleTasksConcurrent(t *testing.T) { + ctx := context.Background() + manager := NewExternalCollectionManager(ctx, 4) + defer manager.Close() + + clusterID := "test-cluster" + numTasks := 10 + + // Submit multiple tasks concurrently + for i := 0; i < numTasks; i++ { + taskID := int64(i + 100) + collID := int64(i + 1000) + + req := &datapb.UpdateExternalCollectionRequest{ + TaskID: taskID, + CollectionID: collID, + } + + taskFunc := func(ctx context.Context) (*datapb.UpdateExternalCollectionResponse, error) { + return &datapb.UpdateExternalCollectionResponse{ + State: indexpb.JobState_JobStateFinished, + }, nil + } + + err := manager.SubmitTask(clusterID, req, taskFunc) + assert.NoError(t, err) + } + + require.Eventually(t, func() bool { + for i := 0; i < numTasks; i++ { + taskID := int64(i + 100) + info := manager.Get(clusterID, taskID) + if info == nil || info.State != indexpb.JobState_JobStateFinished { + return false + } + } + return true + }, time.Second, 10*time.Millisecond) + + // Tasks remain queryable until dropped + for i := 0; i < numTasks; i++ { + taskID := int64(i + 100) + info := manager.Get(clusterID, taskID) + assert.NotNil(t, info) + assert.Equal(t, indexpb.JobState_JobStateFinished, info.State) + } +} + +func TestExternalCollectionManager_Close(t *testing.T) { + ctx := context.Background() + manager := NewExternalCollectionManager(ctx, 4) + + clusterID := "test-cluster" + taskID := int64(5) + collID := int64(500) + + req := &datapb.UpdateExternalCollectionRequest{ + TaskID: taskID, + CollectionID: collID, + } + + // Submit a task + var executed atomic.Bool + started := make(chan struct{}) + unblock := make(chan struct{}) + taskFunc := func(ctx context.Context) (*datapb.UpdateExternalCollectionResponse, error) { + close(started) + select { + case <-unblock: + case <-ctx.Done(): + return nil, ctx.Err() + } + executed.Store(true) + return &datapb.UpdateExternalCollectionResponse{ + State: indexpb.JobState_JobStateFinished, + }, nil + } + + err := manager.SubmitTask(clusterID, req, taskFunc) + assert.NoError(t, err) + + require.Eventually(t, func() bool { + select { + case <-started: + return true + default: + return false + } + }, time.Second, 10*time.Millisecond) + + // Close manager while the task is still running + manager.Close() + + close(unblock) + + require.Eventually(t, func() bool { + return executed.Load() + }, time.Second, 10*time.Millisecond) + + // Task should have executed before close + assert.True(t, executed.Load()) +} + +func TestExternalCollectionManager_UpdateStateNonExistent(t *testing.T) { + ctx := context.Background() + manager := NewExternalCollectionManager(ctx, 4) + defer manager.Close() + + clusterID := "test-cluster" + taskID := int64(999) + + // Try to update state of non-existent task (should not panic) + manager.UpdateState(clusterID, taskID, indexpb.JobState_JobStateFinished, "") + + // Get should return nil + info := manager.Get(clusterID, taskID) + assert.Nil(t, info) +} + +func TestExternalCollectionManager_DeleteNonExistent(t *testing.T) { + ctx := context.Background() + manager := NewExternalCollectionManager(ctx, 4) + defer manager.Close() + + clusterID := "test-cluster" + taskID := int64(888) + + // Try to delete non-existent task + info := manager.Delete(clusterID, taskID) + assert.Nil(t, info) +} diff --git a/internal/datanode/external/task_update.go b/internal/datanode/external/task_update.go new file mode 100644 index 0000000000..9d54558786 --- /dev/null +++ b/internal/datanode/external/task_update.go @@ -0,0 +1,456 @@ +// 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 external + +import ( + "context" + "fmt" + "sort" + + "go.uber.org/zap" + + "github.com/milvus-io/milvus/pkg/v2/log" + "github.com/milvus-io/milvus/pkg/v2/proto/datapb" + "github.com/milvus-io/milvus/pkg/v2/proto/indexpb" + "github.com/milvus-io/milvus/pkg/v2/util/timerecord" +) + +func ensureContext(ctx context.Context) error { + if ctx == nil { + return nil + } + select { + case <-ctx.Done(): + return ctx.Err() + default: + return nil + } +} + +// Fragment represents a data fragment from external source +type Fragment struct { + FragmentID int64 + RowCount int64 +} + +// FragmentRowRange represents the row index range for a fragment within a segment +type FragmentRowRange struct { + FragmentID int64 + StartRow int64 // inclusive + EndRow int64 // exclusive +} + +// SegmentRowMapping holds the row index mapping for all fragments in a segment +type SegmentRowMapping struct { + SegmentID int64 + TotalRows int64 + Ranges []FragmentRowRange + Fragments []Fragment +} + +// NewSegmentRowMapping creates a row mapping from fragments +// Fragments are mapped sequentially: fragment1 gets [0, rowCount1), fragment2 gets [rowCount1, rowCount1+rowCount2), etc. +func NewSegmentRowMapping(segmentID int64, fragments []Fragment) *SegmentRowMapping { + mapping := &SegmentRowMapping{ + SegmentID: segmentID, + Fragments: fragments, + Ranges: make([]FragmentRowRange, len(fragments)), + } + + var offset int64 + for i, f := range fragments { + mapping.Ranges[i] = FragmentRowRange{ + FragmentID: f.FragmentID, + StartRow: offset, + EndRow: offset + f.RowCount, + } + offset += f.RowCount + } + mapping.TotalRows = offset + + return mapping +} + +// GetFragmentByRowIndex returns the fragment range that contains the given row index +// Returns nil if rowIndex is out of range +// To get local index within fragment: rowIndex - range.StartRow +func (m *SegmentRowMapping) GetFragmentByRowIndex(rowIndex int64) *FragmentRowRange { + if rowIndex < 0 || rowIndex >= m.TotalRows { + return nil + } + + // Binary search for efficiency + left, right := 0, len(m.Ranges)-1 + for left <= right { + mid := (left + right) / 2 + r := &m.Ranges[mid] + if rowIndex >= r.StartRow && rowIndex < r.EndRow { + return r + } else if rowIndex < r.StartRow { + right = mid - 1 + } else { + left = mid + 1 + } + } + return nil +} + +// SegmentFragments maps segment ID to its fragments +type SegmentFragments map[int64][]Fragment + +// SegmentResult holds a segment and its fragment row mapping +type SegmentResult struct { + Segment *datapb.SegmentInfo + RowMapping *SegmentRowMapping +} + +// UpdateExternalTask handles updating external collection segments +type UpdateExternalTask struct { + ctx context.Context + cancel context.CancelFunc + + req *datapb.UpdateExternalCollectionRequest + tr *timerecord.TimeRecorder + + state indexpb.JobState + failReason string + + // Result after execution + updatedSegments []*datapb.SegmentInfo + segmentMappings map[int64]*SegmentRowMapping // segmentID -> row mapping +} + +// NewUpdateExternalTask creates a new update external task +func NewUpdateExternalTask( + ctx context.Context, + cancel context.CancelFunc, + req *datapb.UpdateExternalCollectionRequest, +) *UpdateExternalTask { + return &UpdateExternalTask{ + ctx: ctx, + cancel: cancel, + req: req, + tr: timerecord.NewTimeRecorder(fmt.Sprintf("UpdateExternalTask: %d", req.GetTaskID())), + state: indexpb.JobState_JobStateInit, + segmentMappings: make(map[int64]*SegmentRowMapping), + } +} + +func (t *UpdateExternalTask) Ctx() context.Context { + return t.ctx +} + +func (t *UpdateExternalTask) Name() string { + return fmt.Sprintf("UpdateExternalTask-%d", t.req.GetTaskID()) +} + +func (t *UpdateExternalTask) OnEnqueue(ctx context.Context) error { + t.tr.RecordSpan() + log.Ctx(ctx).Info("UpdateExternalTask enqueued", + zap.Int64("taskID", t.req.GetTaskID()), + zap.Int64("collectionID", t.req.GetCollectionID())) + return nil +} + +func (t *UpdateExternalTask) SetState(state indexpb.JobState, failReason string) { + t.state = state + t.failReason = failReason +} + +func (t *UpdateExternalTask) GetState() indexpb.JobState { + return t.state +} + +func (t *UpdateExternalTask) GetSlot() int64 { + return 1 +} + +func (t *UpdateExternalTask) Reset() { + t.ctx = nil + t.cancel = nil + t.req = nil + t.tr = nil + t.updatedSegments = nil + t.segmentMappings = nil +} + +func (t *UpdateExternalTask) PreExecute(ctx context.Context) error { + if err := ensureContext(ctx); err != nil { + return err + } + log.Ctx(ctx).Info("UpdateExternalTask PreExecute", + zap.Int64("taskID", t.req.GetTaskID()), + zap.Int64("collectionID", t.req.GetCollectionID())) + + if t.req == nil { + return fmt.Errorf("request is nil") + } + return nil +} + +func (t *UpdateExternalTask) Execute(ctx context.Context) error { + if err := ensureContext(ctx); err != nil { + return err + } + log.Ctx(ctx).Info("UpdateExternalTask Execute", + zap.Int64("taskID", t.req.GetTaskID()), + zap.Int64("collectionID", t.req.GetCollectionID())) + + // TODO: Fetch fragments from external source + // newFragments := fetchFragmentsFromExternalSource(t.req.GetExternalSource(), t.req.GetExternalSpec()) + var newFragments []Fragment + + // Build current segment -> fragments mapping + // TODO: This mapping should come from metadata or be stored in SegmentInfo + currentSegmentFragments := t.buildCurrentSegmentFragments() + + // Compare and organize segments + updatedSegments, err := t.organizeSegments(ctx, currentSegmentFragments, newFragments) + if err != nil { + return err + } + + t.updatedSegments = updatedSegments + return nil +} + +func (t *UpdateExternalTask) PostExecute(ctx context.Context) error { + if err := ensureContext(ctx); err != nil { + return err + } + log.Ctx(ctx).Info("UpdateExternalTask PostExecute", + zap.Int64("taskID", t.req.GetTaskID()), + zap.Int64("collectionID", t.req.GetCollectionID()), + zap.Int("updatedSegments", len(t.updatedSegments))) + return nil +} + +// GetUpdatedSegments returns the result segments after execution +func (t *UpdateExternalTask) GetUpdatedSegments() []*datapb.SegmentInfo { + return t.updatedSegments +} + +// GetSegmentMappings returns the row mappings for all segments (segmentID -> mapping) +func (t *UpdateExternalTask) GetSegmentMappings() map[int64]*SegmentRowMapping { + return t.segmentMappings +} + +// buildCurrentSegmentFragments builds segment to fragments mapping from current segments +func (t *UpdateExternalTask) buildCurrentSegmentFragments() SegmentFragments { + result := make(SegmentFragments) + // TODO: Extract fragment information from segment metadata + // For now, this is a placeholder - fragment info should be stored in segment metadata + for _, seg := range t.req.GetCurrentSegments() { + // Placeholder: each segment has its own "virtual" fragment + result[seg.GetID()] = []Fragment{ + {FragmentID: seg.GetID(), RowCount: seg.GetNumOfRows()}, + } + } + return result +} + +// organizeSegments compares fragments and organizes them into segments +func (t *UpdateExternalTask) organizeSegments( + ctx context.Context, + currentSegmentFragments SegmentFragments, + newFragments []Fragment, +) ([]*datapb.SegmentInfo, error) { + if err := ensureContext(ctx); err != nil { + return nil, err + } + log := log.Ctx(ctx) + + // Build new fragment map for quick lookup + newFragmentMap := make(map[int64]Fragment) + for _, f := range newFragments { + newFragmentMap[f.FragmentID] = f + } + + // Track which fragments are used by kept segments + usedFragments := make(map[int64]bool) + var keptSegments []*datapb.SegmentInfo + + // Check each current segment + for _, seg := range t.req.GetCurrentSegments() { + if err := ensureContext(ctx); err != nil { + return nil, err + } + fragments := currentSegmentFragments[seg.GetID()] + allFragmentsExist := true + + // Check if all fragments of this segment still exist + for _, f := range fragments { + if _, exists := newFragmentMap[f.FragmentID]; !exists { + allFragmentsExist = false + log.Info("Fragment removed from segment", + zap.Int64("segmentID", seg.GetID()), + zap.Int64("fragmentID", f.FragmentID)) + break + } + } + + if allFragmentsExist { + // Keep this segment unchanged + keptSegments = append(keptSegments, seg) + for _, f := range fragments { + if err := ensureContext(ctx); err != nil { + return nil, err + } + usedFragments[f.FragmentID] = true + } + // Compute row mapping for kept segment + t.segmentMappings[seg.GetID()] = NewSegmentRowMapping(seg.GetID(), fragments) + log.Debug("Segment kept unchanged", + zap.Int64("segmentID", seg.GetID())) + } else { + // Segment invalidated - its remaining fragments become orphans + log.Info("Segment invalidated due to removed fragments", + zap.Int64("segmentID", seg.GetID())) + } + } + + // Collect orphan fragments (new + from invalidated segments) + var orphanFragments []Fragment + for _, f := range newFragments { + if err := ensureContext(ctx); err != nil { + return nil, err + } + if !usedFragments[f.FragmentID] { + orphanFragments = append(orphanFragments, f) + } + } + + // Organize orphan fragments into new segments with balanced row counts + newSegments, err := t.balanceFragmentsToSegments(ctx, orphanFragments) + if err != nil { + return nil, err + } + + // Combine kept and new segments + result := append(keptSegments, newSegments...) + + log.Info("Segment organization complete", + zap.Int("keptSegments", len(keptSegments)), + zap.Int("newSegments", len(newSegments)), + zap.Int("totalSegments", len(result))) + + return result, nil +} + +// balanceFragmentsToSegments organizes fragments into segments with balanced row counts +func (t *UpdateExternalTask) balanceFragmentsToSegments(ctx context.Context, fragments []Fragment) ([]*datapb.SegmentInfo, error) { + if len(fragments) == 0 { + return nil, nil + } + if err := ensureContext(ctx); err != nil { + return nil, err + } + + log := log.Ctx(ctx) + + // Calculate total rows + var totalRows int64 + for _, f := range fragments { + if err := ensureContext(ctx); err != nil { + return nil, err + } + totalRows += f.RowCount + } + + // Determine target rows per segment (use a reasonable default) + // TODO: Make this configurable or based on segment size limits + targetRowsPerSegment := int64(1000000) // 1M rows per segment as default + if totalRows < targetRowsPerSegment { + targetRowsPerSegment = totalRows + } + + numSegments := (totalRows + targetRowsPerSegment - 1) / targetRowsPerSegment + if numSegments == 0 { + numSegments = 1 + } + + avgRowsPerSegment := totalRows / numSegments + + log.Info("Balancing fragments to segments", + zap.Int("numFragments", len(fragments)), + zap.Int64("totalRows", totalRows), + zap.Int64("numSegments", numSegments), + zap.Int64("avgRowsPerSegment", avgRowsPerSegment)) + + // Sort fragments by row count descending for better bin-packing + sortedFragments := make([]Fragment, len(fragments)) + copy(sortedFragments, fragments) + sort.Slice(sortedFragments, func(i, j int) bool { + return sortedFragments[i].RowCount > sortedFragments[j].RowCount + }) + + // Initialize segment bins + type segmentBin struct { + fragments []Fragment + rowCount int64 + } + bins := make([]segmentBin, numSegments) + + // Greedy bin-packing: assign each fragment to the bin with lowest current row count + for _, f := range sortedFragments { + if err := ensureContext(ctx); err != nil { + return nil, err + } + // Find bin with minimum row count + minIdx := 0 + for i := 1; i < len(bins); i++ { + if bins[i].rowCount < bins[minIdx].rowCount { + minIdx = i + } + } + bins[minIdx].fragments = append(bins[minIdx].fragments, f) + bins[minIdx].rowCount += f.RowCount + } + + // Convert bins to SegmentInfo + var result []*datapb.SegmentInfo + for i, bin := range bins { + if err := ensureContext(ctx); err != nil { + return nil, err + } + if len(bin.fragments) == 0 { + continue + } + + // TODO: Generate column groups + // Just placeholder here. ID will be assigned by coordinator. + segmentID := int64(i + 1) + + seg := &datapb.SegmentInfo{ + ID: segmentID, + CollectionID: t.req.GetCollectionID(), + NumOfRows: bin.rowCount, + // TODO: Fill other required fields + } + result = append(result, seg) + + // Compute and store row mapping for new segment + t.segmentMappings[segmentID] = NewSegmentRowMapping(segmentID, bin.fragments) + + log.Debug("Created new segment from fragments", + zap.Int64("segmentID placeholder", segmentID), + zap.Int64("rowCount", bin.rowCount), + zap.Int("numFragments", len(bin.fragments))) + } + + return result, nil +} diff --git a/internal/datanode/external/task_update_test.go b/internal/datanode/external/task_update_test.go new file mode 100644 index 0000000000..bb9ffd4a06 --- /dev/null +++ b/internal/datanode/external/task_update_test.go @@ -0,0 +1,536 @@ +// 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 external + +import ( + "context" + "testing" + + "github.com/stretchr/testify/suite" + + "github.com/milvus-io/milvus/pkg/v2/proto/datapb" + "github.com/milvus-io/milvus/pkg/v2/proto/indexpb" +) + +type UpdateExternalTaskSuite struct { + suite.Suite + collectionID int64 + taskID int64 +} + +func (s *UpdateExternalTaskSuite) SetupSuite() { + s.collectionID = 1000 + s.taskID = 1 +} + +func (s *UpdateExternalTaskSuite) TestNewUpdateExternalTask() { + ctx, cancel := context.WithCancel(context.Background()) + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: s.collectionID, + TaskID: s.taskID, + ExternalSource: "test_source", + ExternalSpec: "test_spec", + } + + task := NewUpdateExternalTask(ctx, cancel, req) + + s.NotNil(task) + s.Equal(s.collectionID, task.req.GetCollectionID()) + s.Equal(s.taskID, task.req.GetTaskID()) + s.Equal(indexpb.JobState_JobStateInit, task.GetState()) + s.Contains(task.Name(), "UpdateExternalTask") +} + +func (s *UpdateExternalTaskSuite) TestTaskLifecycle() { + ctx, cancel := context.WithCancel(context.Background()) + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: s.collectionID, + TaskID: s.taskID, + ExternalSource: "test_source", + CurrentSegments: []*datapb.SegmentInfo{ + {ID: 1, CollectionID: s.collectionID, NumOfRows: 1000}, + {ID: 2, CollectionID: s.collectionID, NumOfRows: 2000}, + }, + } + + task := NewUpdateExternalTask(ctx, cancel, req) + + // Test OnEnqueue + err := task.OnEnqueue(ctx) + s.NoError(err) + + // Test PreExecute + err = task.PreExecute(ctx) + s.NoError(err) + + // Test Execute + err = task.Execute(ctx) + s.NoError(err) + + // Test PostExecute + err = task.PostExecute(ctx) + s.NoError(err) + + // Test GetSlot + s.Equal(int64(1), task.GetSlot()) +} + +func (s *UpdateExternalTaskSuite) TestPreExecuteWithNilRequest() { + ctx, cancel := context.WithCancel(context.Background()) + task := &UpdateExternalTask{ + ctx: ctx, + cancel: cancel, + req: nil, + } + + err := task.PreExecute(ctx) + s.Error(err) +} + +func (s *UpdateExternalTaskSuite) TestSetAndGetState() { + ctx, cancel := context.WithCancel(context.Background()) + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: s.collectionID, + TaskID: s.taskID, + } + + task := NewUpdateExternalTask(ctx, cancel, req) + + task.SetState(indexpb.JobState_JobStateInProgress, "") + s.Equal(indexpb.JobState_JobStateInProgress, task.GetState()) + + task.SetState(indexpb.JobState_JobStateFailed, "test failure") + s.Equal(indexpb.JobState_JobStateFailed, task.GetState()) + s.Equal("test failure", task.failReason) +} + +func (s *UpdateExternalTaskSuite) TestReset() { + ctx, cancel := context.WithCancel(context.Background()) + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: s.collectionID, + TaskID: s.taskID, + } + + task := NewUpdateExternalTask(ctx, cancel, req) + task.Reset() + + s.Nil(task.ctx) + s.Nil(task.cancel) + s.Nil(task.req) + s.Nil(task.tr) + s.Nil(task.updatedSegments) +} + +func (s *UpdateExternalTaskSuite) TestBalanceFragmentsToSegments_Empty() { + ctx, cancel := context.WithCancel(context.Background()) + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: s.collectionID, + TaskID: s.taskID, + } + + task := NewUpdateExternalTask(ctx, cancel, req) + result, err := task.balanceFragmentsToSegments(context.Background(), []Fragment{}) + s.NoError(err) + s.Nil(result) +} + +func (s *UpdateExternalTaskSuite) TestBalanceFragmentsToSegments_SingleFragment() { + ctx, cancel := context.WithCancel(context.Background()) + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: s.collectionID, + TaskID: s.taskID, + } + + task := NewUpdateExternalTask(ctx, cancel, req) + fragments := []Fragment{ + {FragmentID: 1, RowCount: 500}, + } + + result, err := task.balanceFragmentsToSegments(context.Background(), fragments) + s.NoError(err) + s.Len(result, 1) + s.Equal(int64(500), result[0].GetNumOfRows()) +} + +func (s *UpdateExternalTaskSuite) TestBalanceFragmentsToSegments_MultipleFragments() { + ctx, cancel := context.WithCancel(context.Background()) + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: s.collectionID, + TaskID: s.taskID, + } + + task := NewUpdateExternalTask(ctx, cancel, req) + fragments := []Fragment{ + {FragmentID: 1, RowCount: 300000}, + {FragmentID: 2, RowCount: 400000}, + {FragmentID: 3, RowCount: 500000}, + {FragmentID: 4, RowCount: 600000}, + {FragmentID: 5, RowCount: 200000}, + } + + result, err := task.balanceFragmentsToSegments(context.Background(), fragments) + s.NoError(err) + + // Verify total rows are preserved + var totalRows int64 + for _, seg := range result { + totalRows += seg.GetNumOfRows() + } + s.Equal(int64(2000000), totalRows) + + // Verify segments are reasonably balanced + if len(result) > 1 { + var minRows, maxRows int64 = result[0].GetNumOfRows(), result[0].GetNumOfRows() + for _, seg := range result { + if seg.GetNumOfRows() < minRows { + minRows = seg.GetNumOfRows() + } + if seg.GetNumOfRows() > maxRows { + maxRows = seg.GetNumOfRows() + } + } + // The difference between max and min should be reasonable + // (less than 2x the average fragment size) + avgFragmentSize := int64(2000000 / 5) + s.Less(maxRows-minRows, avgFragmentSize*2) + } +} + +func (s *UpdateExternalTaskSuite) TestPreExecuteContextCanceled() { + ctx, cancel := context.WithCancel(context.Background()) + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: s.collectionID, + TaskID: s.taskID, + } + + task := NewUpdateExternalTask(ctx, cancel, req) + cancel() + + err := task.PreExecute(ctx) + s.ErrorIs(err, context.Canceled) +} + +func (s *UpdateExternalTaskSuite) TestExecuteContextCanceled() { + ctx, cancel := context.WithCancel(context.Background()) + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: s.collectionID, + TaskID: s.taskID, + } + + task := NewUpdateExternalTask(ctx, cancel, req) + cancel() + + err := task.Execute(ctx) + s.ErrorIs(err, context.Canceled) +} + +func (s *UpdateExternalTaskSuite) TestBalanceFragmentsToSegmentsContextCanceled() { + ctx, cancel := context.WithCancel(context.Background()) + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: s.collectionID, + TaskID: s.taskID, + } + + task := NewUpdateExternalTask(ctx, cancel, req) + cancel() + + result, err := task.balanceFragmentsToSegments(ctx, []Fragment{{FragmentID: 1, RowCount: 10}}) + s.ErrorIs(err, context.Canceled) + s.Nil(result) +} + +func (s *UpdateExternalTaskSuite) TestOrganizeSegments_AllFragmentsExist() { + ctx, cancel := context.WithCancel(context.Background()) + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: s.collectionID, + TaskID: s.taskID, + CurrentSegments: []*datapb.SegmentInfo{ + {ID: 1, CollectionID: s.collectionID, NumOfRows: 1000}, + {ID: 2, CollectionID: s.collectionID, NumOfRows: 2000}, + }, + } + + task := NewUpdateExternalTask(ctx, cancel, req) + + // Simulate current segment fragments mapping + currentSegmentFragments := SegmentFragments{ + 1: []Fragment{{FragmentID: 101, RowCount: 1000}}, + 2: []Fragment{{FragmentID: 102, RowCount: 2000}}, + } + + // New fragments contain all existing fragments + newFragments := []Fragment{ + {FragmentID: 101, RowCount: 1000}, + {FragmentID: 102, RowCount: 2000}, + } + + result, err := task.organizeSegments(context.Background(), currentSegmentFragments, newFragments) + s.NoError(err) + + // Both segments should be kept + s.Len(result, 2) +} + +func (s *UpdateExternalTaskSuite) TestOrganizeSegments_FragmentRemoved() { + ctx, cancel := context.WithCancel(context.Background()) + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: s.collectionID, + TaskID: s.taskID, + CurrentSegments: []*datapb.SegmentInfo{ + {ID: 1, CollectionID: s.collectionID, NumOfRows: 1000}, + {ID: 2, CollectionID: s.collectionID, NumOfRows: 2000}, + }, + } + + task := NewUpdateExternalTask(ctx, cancel, req) + + // Segment 1 has fragment 101, Segment 2 has fragments 102 and 103 + currentSegmentFragments := SegmentFragments{ + 1: []Fragment{{FragmentID: 101, RowCount: 1000}}, + 2: []Fragment{{FragmentID: 102, RowCount: 1000}, {FragmentID: 103, RowCount: 1000}}, + } + + // Fragment 103 is removed - segment 2 should be invalidated + newFragments := []Fragment{ + {FragmentID: 101, RowCount: 1000}, + {FragmentID: 102, RowCount: 1000}, + } + + result, err := task.organizeSegments(context.Background(), currentSegmentFragments, newFragments) + s.NoError(err) + + // Segment 1 should be kept, segment 2 invalidated, fragment 102 becomes orphan + // Result should have segment 1 kept + new segment for orphan fragment 102 + s.GreaterOrEqual(len(result), 1) + + // Verify segment 1 is in the result + hasSegment1 := false + for _, seg := range result { + if seg.GetID() == 1 { + hasSegment1 = true + break + } + } + s.True(hasSegment1, "Segment 1 should be kept") +} + +func (s *UpdateExternalTaskSuite) TestOrganizeSegments_NewFragmentsAdded() { + ctx, cancel := context.WithCancel(context.Background()) + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: s.collectionID, + TaskID: s.taskID, + CurrentSegments: []*datapb.SegmentInfo{ + {ID: 1, CollectionID: s.collectionID, NumOfRows: 1000}, + }, + } + + task := NewUpdateExternalTask(ctx, cancel, req) + + currentSegmentFragments := SegmentFragments{ + 1: []Fragment{{FragmentID: 101, RowCount: 1000}}, + } + + // New fragments include existing + new ones + newFragments := []Fragment{ + {FragmentID: 101, RowCount: 1000}, + {FragmentID: 102, RowCount: 2000}, // new + {FragmentID: 103, RowCount: 3000}, // new + } + + result, err := task.organizeSegments(context.Background(), currentSegmentFragments, newFragments) + s.NoError(err) + + // Should have segment 1 kept + new segments for orphan fragments + s.GreaterOrEqual(len(result), 2) + + // Verify total rows + var totalRows int64 + for _, seg := range result { + totalRows += seg.GetNumOfRows() + } + s.Equal(int64(6000), totalRows) +} + +func (s *UpdateExternalTaskSuite) TestNewSegmentRowMapping() { + fragments := []Fragment{ + {FragmentID: 1, RowCount: 100}, + {FragmentID: 2, RowCount: 200}, + {FragmentID: 3, RowCount: 150}, + } + + mapping := NewSegmentRowMapping(1001, fragments) + + s.Equal(int64(1001), mapping.SegmentID) + s.Equal(int64(450), mapping.TotalRows) + s.Len(mapping.Ranges, 3) + + // Check ranges + s.Equal(int64(1), mapping.Ranges[0].FragmentID) + s.Equal(int64(0), mapping.Ranges[0].StartRow) + s.Equal(int64(100), mapping.Ranges[0].EndRow) + + s.Equal(int64(2), mapping.Ranges[1].FragmentID) + s.Equal(int64(100), mapping.Ranges[1].StartRow) + s.Equal(int64(300), mapping.Ranges[1].EndRow) + + s.Equal(int64(3), mapping.Ranges[2].FragmentID) + s.Equal(int64(300), mapping.Ranges[2].StartRow) + s.Equal(int64(450), mapping.Ranges[2].EndRow) +} + +func (s *UpdateExternalTaskSuite) TestGetFragmentByRowIndex() { + fragments := []Fragment{ + {FragmentID: 1, RowCount: 100}, + {FragmentID: 2, RowCount: 200}, + {FragmentID: 3, RowCount: 150}, + } + mapping := NewSegmentRowMapping(1001, fragments) + + // Test first fragment + r := mapping.GetFragmentByRowIndex(0) + s.NotNil(r) + s.Equal(int64(1), r.FragmentID) + + r = mapping.GetFragmentByRowIndex(99) + s.NotNil(r) + s.Equal(int64(1), r.FragmentID) + + // Test second fragment + r = mapping.GetFragmentByRowIndex(100) + s.NotNil(r) + s.Equal(int64(2), r.FragmentID) + + r = mapping.GetFragmentByRowIndex(299) + s.NotNil(r) + s.Equal(int64(2), r.FragmentID) + + // Test third fragment + r = mapping.GetFragmentByRowIndex(300) + s.NotNil(r) + s.Equal(int64(3), r.FragmentID) + + r = mapping.GetFragmentByRowIndex(449) + s.NotNil(r) + s.Equal(int64(3), r.FragmentID) + + // Test out of range + r = mapping.GetFragmentByRowIndex(-1) + s.Nil(r) + + r = mapping.GetFragmentByRowIndex(450) + s.Nil(r) + + r = mapping.GetFragmentByRowIndex(1000) + s.Nil(r) +} + +func (s *UpdateExternalTaskSuite) TestGetFragmentByRowIndex_LocalIndex() { + fragments := []Fragment{ + {FragmentID: 1, RowCount: 100}, + {FragmentID: 2, RowCount: 200}, + } + mapping := NewSegmentRowMapping(1001, fragments) + + // Row 0 -> fragment 1, local index 0 + r := mapping.GetFragmentByRowIndex(0) + s.NotNil(r) + s.Equal(int64(1), r.FragmentID) + s.Equal(int64(0), 0-r.StartRow) // local index + + // Row 50 -> fragment 1, local index 50 + r = mapping.GetFragmentByRowIndex(50) + s.NotNil(r) + s.Equal(int64(1), r.FragmentID) + s.Equal(int64(50), 50-r.StartRow) + + // Row 100 -> fragment 2, local index 0 + r = mapping.GetFragmentByRowIndex(100) + s.NotNil(r) + s.Equal(int64(2), r.FragmentID) + s.Equal(int64(0), 100-r.StartRow) + + // Row 150 -> fragment 2, local index 50 + r = mapping.GetFragmentByRowIndex(150) + s.NotNil(r) + s.Equal(int64(2), r.FragmentID) + s.Equal(int64(50), 150-r.StartRow) + + // Row 299 -> fragment 2, local index 199 + r = mapping.GetFragmentByRowIndex(299) + s.NotNil(r) + s.Equal(int64(2), r.FragmentID) + s.Equal(int64(199), 299-r.StartRow) +} + +func (s *UpdateExternalTaskSuite) TestSegmentRowMapping_EmptyFragments() { + mapping := NewSegmentRowMapping(1001, []Fragment{}) + + s.Equal(int64(0), mapping.TotalRows) + s.Len(mapping.Ranges, 0) + + r := mapping.GetFragmentByRowIndex(0) + s.Nil(r) +} + +func (s *UpdateExternalTaskSuite) TestMappingsComputedDuringOrganize() { + ctx, cancel := context.WithCancel(context.Background()) + // Use segment ID 100 to avoid collision with placeholder ID (1) + req := &datapb.UpdateExternalCollectionRequest{ + CollectionID: s.collectionID, + TaskID: s.taskID, + CurrentSegments: []*datapb.SegmentInfo{ + {ID: 100, CollectionID: s.collectionID, NumOfRows: 1000}, + }, + } + + task := NewUpdateExternalTask(ctx, cancel, req) + + // Simulate current segment has fragment 101 + currentSegmentFragments := SegmentFragments{ + 100: []Fragment{{FragmentID: 101, RowCount: 1000}}, + } + + // New fragments include existing + new ones + newFragments := []Fragment{ + {FragmentID: 101, RowCount: 1000}, + {FragmentID: 102, RowCount: 500}, + } + + _, err := task.organizeSegments(context.Background(), currentSegmentFragments, newFragments) + s.NoError(err) + + mappings := task.GetSegmentMappings() + s.Len(mappings, 2) + + // Check mapping for kept segment (ID=100) + mapping100 := mappings[100] + s.NotNil(mapping100) + s.Equal(int64(1000), mapping100.TotalRows) + s.Len(mapping100.Ranges, 1) + s.Equal(int64(101), mapping100.Ranges[0].FragmentID) + + // Check mapping for new segment (ID=1, placeholder) + mapping1 := mappings[1] + s.NotNil(mapping1) + s.Equal(int64(500), mapping1.TotalRows) + s.Len(mapping1.Ranges, 1) + s.Equal(int64(102), mapping1.Ranges[0].FragmentID) +} + +func TestUpdateExternalTaskSuite(t *testing.T) { + suite.Run(t, new(UpdateExternalTaskSuite)) +} diff --git a/internal/datanode/services.go b/internal/datanode/services.go index ff4b0e030e..797bb0529c 100644 --- a/internal/datanode/services.go +++ b/internal/datanode/services.go @@ -30,6 +30,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/compaction" "github.com/milvus-io/milvus/internal/datanode/compactor" + "github.com/milvus-io/milvus/internal/datanode/external" "github.com/milvus-io/milvus/internal/datanode/importv2" "github.com/milvus-io/milvus/internal/datanode/index" "github.com/milvus-io/milvus/internal/flushcommon/io" @@ -40,6 +41,7 @@ import ( "github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/metrics" "github.com/milvus-io/milvus/pkg/v2/proto/datapb" + "github.com/milvus-io/milvus/pkg/v2/proto/indexpb" "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "github.com/milvus-io/milvus/pkg/v2/proto/workerpb" "github.com/milvus-io/milvus/pkg/v2/taskcommon" @@ -619,6 +621,12 @@ func (node *DataNode) CreateTask(ctx context.Context, request *workerpb.CreateTa return merr.Status(err), nil } return node.createAnalyzeTask(ctx, req) + case taskcommon.ExternalCollection: + req := &datapb.UpdateExternalCollectionRequest{} + if err := proto.Unmarshal(request.GetPayload(), req); err != nil { + return merr.Status(err), nil + } + return node.createExternalCollectionTask(ctx, req) default: err := fmt.Errorf("unrecognized task type '%s', properties=%v", taskType, request.GetProperties()) log.Ctx(ctx).Warn("CreateTask failed", zap.Error(err)) @@ -729,6 +737,31 @@ func (node *DataNode) QueryTask(ctx context.Context, request *workerpb.QueryTask resProperties.AppendReason(results[0].GetFailReason()) } return wrapQueryTaskResult(resp, resProperties) + case taskcommon.ExternalCollection: + // Query task state from external collection manager + info := node.externalCollectionManager.Get(clusterID, taskID) + if info == nil { + resp := &datapb.UpdateExternalCollectionResponse{ + Status: merr.Success(), + State: indexpb.JobState_JobStateFailed, + FailReason: "task result not found", + } + resProperties := taskcommon.NewProperties(nil) + resProperties.AppendTaskState(taskcommon.Failed) + resProperties.AppendReason("task result not found") + return wrapQueryTaskResult(resp, resProperties) + } + resp := &datapb.UpdateExternalCollectionResponse{ + Status: merr.Success(), + State: info.State, + FailReason: info.FailReason, + KeptSegments: info.KeptSegments, + UpdatedSegments: info.UpdatedSegments, + } + resProperties := taskcommon.NewProperties(nil) + resProperties.AppendTaskState(info.State) + resProperties.AppendReason(info.FailReason) + return wrapQueryTaskResult(resp, resProperties) default: err := fmt.Errorf("unrecognized task type '%s', properties=%v", taskType, request.GetProperties()) log.Ctx(ctx).Warn("QueryTask failed", zap.Error(err)) @@ -772,6 +805,21 @@ func (node *DataNode) DropTask(ctx context.Context, request *workerpb.DropTaskRe TaskIDs: []int64{taskID}, JobType: jobType, }) + case taskcommon.ExternalCollection: + // Drop external collection task from external collection manager + clusterID, err := properties.GetClusterID() + if err != nil { + return merr.Status(err), nil + } + cancelled := node.externalCollectionManager.CancelTask(clusterID, taskID) + info := node.externalCollectionManager.Delete(clusterID, taskID) + if !cancelled && info != nil && info.Cancel != nil { + info.Cancel() + } + log.Ctx(ctx).Info("DropTask for external collection completed", + zap.Int64("taskID", taskID), + zap.String("clusterID", clusterID)) + return merr.Success(), nil default: err := fmt.Errorf("unrecognized task type '%s', properties=%v", taskType, request.GetProperties()) log.Ctx(ctx).Warn("DropTask failed", zap.Error(err)) @@ -794,3 +842,77 @@ func (node *DataNode) SyncFileResource(ctx context.Context, req *internalpb.Sync } return merr.Success(), nil } + +// createExternalCollectionTask handles updating external collection segments +// This submits the task to the external collection manager for async execution +func (node *DataNode) createExternalCollectionTask(ctx context.Context, req *datapb.UpdateExternalCollectionRequest) (*commonpb.Status, error) { + log := log.Ctx(ctx).With( + zap.Int64("taskID", req.GetTaskID()), + zap.Int64("collectionID", req.GetCollectionID()), + ) + + log.Info("createExternalCollectionTask received", + zap.Int("currentSegments", len(req.GetCurrentSegments())), + zap.String("externalSource", req.GetExternalSource())) + + if err := merr.CheckHealthy(node.GetStateCode()); err != nil { + return merr.Status(err), nil + } + + clusterID := paramtable.Get().CommonCfg.ClusterPrefix.GetValue() + + // Submit task to external collection manager + // The task will execute asynchronously in the manager's goroutine pool + err := node.externalCollectionManager.SubmitTask(clusterID, req, func(taskCtx context.Context) (*datapb.UpdateExternalCollectionResponse, error) { + // Execute the task + task := external.NewUpdateExternalTask(taskCtx, func() {}, req) + + if err := task.PreExecute(taskCtx); err != nil { + log.Warn("external collection task PreExecute failed", zap.Error(err)) + return nil, err + } + + if err := task.Execute(taskCtx); err != nil { + log.Warn("external collection task Execute failed", zap.Error(err)) + return nil, err + } + + if err := task.PostExecute(taskCtx); err != nil { + log.Warn("external collection task PostExecute failed", zap.Error(err)) + return nil, err + } + + log.Info("external collection task completed successfully", + zap.Int("updatedSegments", len(task.GetUpdatedSegments()))) + + resp := &datapb.UpdateExternalCollectionResponse{ + Status: merr.Success(), + State: indexpb.JobState_JobStateFinished, + KeptSegments: extractSegmentIDs(req.GetCurrentSegments()), + UpdatedSegments: task.GetUpdatedSegments(), + } + + return resp, nil + }) + if err != nil { + log.Warn("failed to submit external collection task", zap.Error(err)) + return merr.Status(err), nil + } + + log.Info("external collection task submitted to manager") + return merr.Success(), nil +} + +func extractSegmentIDs(segments []*datapb.SegmentInfo) []int64 { + if len(segments) == 0 { + return nil + } + result := make([]int64, 0, len(segments)) + for _, seg := range segments { + if seg == nil { + continue + } + result = append(result, seg.GetID()) + } + return result +} diff --git a/pkg/proto/data_coord.proto b/pkg/proto/data_coord.proto index a52a4100ee..e68ffc171b 100644 --- a/pkg/proto/data_coord.proto +++ b/pkg/proto/data_coord.proto @@ -1148,3 +1148,31 @@ message FileResourceInfo { message CreateExternalCollectionResponse { common.Status status = 1; } + +message UpdateExternalCollectionRequest { + common.MsgBase base = 1; + int64 collectionID = 2; + int64 taskID = 3; + repeated SegmentInfo currentSegments = 4; + string externalSource = 5; + string externalSpec = 6; +} + +message UpdateExternalCollectionResponse { + common.Status status = 1; + repeated int64 keptSegments = 2; + repeated SegmentInfo updatedSegments = 3; + index.JobState state = 4; + string fail_reason = 5; +} + +message QueryExternalCollectionRequest { + int64 taskID = 1; +} + +message DropExternalCollectionRequest { + int64 taskID = 1; +} + +// Note: QueryExternalCollectionRequest and DropExternalCollectionRequest are kept +// for internal use by the unified task system, but are not exposed as separate RPCs diff --git a/pkg/proto/datapb/data_coord.pb.go b/pkg/proto/datapb/data_coord.pb.go index 4960bff02e..9856dd71d9 100644 --- a/pkg/proto/datapb/data_coord.pb.go +++ b/pkg/proto/datapb/data_coord.pb.go @@ -9226,6 +9226,266 @@ func (x *CreateExternalCollectionResponse) GetStatus() *commonpb.Status { return nil } +type UpdateExternalCollectionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + TaskID int64 `protobuf:"varint,3,opt,name=taskID,proto3" json:"taskID,omitempty"` + CurrentSegments []*SegmentInfo `protobuf:"bytes,4,rep,name=currentSegments,proto3" json:"currentSegments,omitempty"` + ExternalSource string `protobuf:"bytes,5,opt,name=externalSource,proto3" json:"externalSource,omitempty"` + ExternalSpec string `protobuf:"bytes,6,opt,name=externalSpec,proto3" json:"externalSpec,omitempty"` +} + +func (x *UpdateExternalCollectionRequest) Reset() { + *x = UpdateExternalCollectionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[109] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateExternalCollectionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateExternalCollectionRequest) ProtoMessage() {} + +func (x *UpdateExternalCollectionRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[109] + 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 UpdateExternalCollectionRequest.ProtoReflect.Descriptor instead. +func (*UpdateExternalCollectionRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{109} +} + +func (x *UpdateExternalCollectionRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *UpdateExternalCollectionRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *UpdateExternalCollectionRequest) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +func (x *UpdateExternalCollectionRequest) GetCurrentSegments() []*SegmentInfo { + if x != nil { + return x.CurrentSegments + } + return nil +} + +func (x *UpdateExternalCollectionRequest) GetExternalSource() string { + if x != nil { + return x.ExternalSource + } + return "" +} + +func (x *UpdateExternalCollectionRequest) GetExternalSpec() string { + if x != nil { + return x.ExternalSpec + } + return "" +} + +type UpdateExternalCollectionResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + KeptSegments []int64 `protobuf:"varint,2,rep,packed,name=keptSegments,proto3" json:"keptSegments,omitempty"` + UpdatedSegments []*SegmentInfo `protobuf:"bytes,3,rep,name=updatedSegments,proto3" json:"updatedSegments,omitempty"` + State indexpb.JobState `protobuf:"varint,4,opt,name=state,proto3,enum=milvus.proto.index.JobState" json:"state,omitempty"` + FailReason string `protobuf:"bytes,5,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"` +} + +func (x *UpdateExternalCollectionResponse) Reset() { + *x = UpdateExternalCollectionResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[110] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateExternalCollectionResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateExternalCollectionResponse) ProtoMessage() {} + +func (x *UpdateExternalCollectionResponse) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[110] + 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 UpdateExternalCollectionResponse.ProtoReflect.Descriptor instead. +func (*UpdateExternalCollectionResponse) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{110} +} + +func (x *UpdateExternalCollectionResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *UpdateExternalCollectionResponse) GetKeptSegments() []int64 { + if x != nil { + return x.KeptSegments + } + return nil +} + +func (x *UpdateExternalCollectionResponse) GetUpdatedSegments() []*SegmentInfo { + if x != nil { + return x.UpdatedSegments + } + return nil +} + +func (x *UpdateExternalCollectionResponse) GetState() indexpb.JobState { + if x != nil { + return x.State + } + return indexpb.JobState(0) +} + +func (x *UpdateExternalCollectionResponse) GetFailReason() string { + if x != nil { + return x.FailReason + } + return "" +} + +type QueryExternalCollectionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TaskID int64 `protobuf:"varint,1,opt,name=taskID,proto3" json:"taskID,omitempty"` +} + +func (x *QueryExternalCollectionRequest) Reset() { + *x = QueryExternalCollectionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[111] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *QueryExternalCollectionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QueryExternalCollectionRequest) ProtoMessage() {} + +func (x *QueryExternalCollectionRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[111] + 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 QueryExternalCollectionRequest.ProtoReflect.Descriptor instead. +func (*QueryExternalCollectionRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{111} +} + +func (x *QueryExternalCollectionRequest) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + +type DropExternalCollectionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + TaskID int64 `protobuf:"varint,1,opt,name=taskID,proto3" json:"taskID,omitempty"` +} + +func (x *DropExternalCollectionRequest) Reset() { + *x = DropExternalCollectionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_data_coord_proto_msgTypes[112] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DropExternalCollectionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DropExternalCollectionRequest) ProtoMessage() {} + +func (x *DropExternalCollectionRequest) ProtoReflect() protoreflect.Message { + mi := &file_data_coord_proto_msgTypes[112] + 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 DropExternalCollectionRequest.ProtoReflect.Descriptor instead. +func (*DropExternalCollectionRequest) Descriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{112} +} + +func (x *DropExternalCollectionRequest) GetTaskID() int64 { + if x != nil { + return x.TaskID + } + return 0 +} + var File_data_coord_proto protoreflect.FileDescriptor var file_data_coord_proto_rawDesc = []byte{ @@ -11015,545 +11275,589 @@ var file_data_coord_proto_rawDesc = []byte{ 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, - 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2a, 0x3e, 0x0a, 0x0b, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x07, 0x0a, 0x03, 0x4e, 0x65, 0x77, 0x10, 0x00, 0x12, - 0x0a, 0x0a, 0x06, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x46, - 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x70, - 0x61, 0x63, 0x74, 0x65, 0x64, 0x10, 0x03, 0x2a, 0x32, 0x0a, 0x0c, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x0a, 0x0a, 0x06, 0x4c, 0x65, 0x67, 0x61, 0x63, - 0x79, 0x10, 0x00, 0x12, 0x06, 0x0a, 0x02, 0x4c, 0x30, 0x10, 0x01, 0x12, 0x06, 0x0a, 0x02, 0x4c, - 0x31, 0x10, 0x02, 0x12, 0x06, 0x0a, 0x02, 0x4c, 0x32, 0x10, 0x03, 0x2a, 0x99, 0x01, 0x0a, 0x11, - 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x55, 0x6e, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x10, - 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x01, 0x12, - 0x0b, 0x0a, 0x07, 0x54, 0x6f, 0x57, 0x61, 0x74, 0x63, 0x68, 0x10, 0x02, 0x12, 0x10, 0x0a, 0x0c, - 0x57, 0x61, 0x74, 0x63, 0x68, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x10, 0x03, 0x12, 0x10, - 0x0a, 0x0c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x10, 0x04, - 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x6f, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x10, 0x05, 0x12, - 0x12, 0x0a, 0x0e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, - 0x73, 0x10, 0x06, 0x12, 0x12, 0x0a, 0x0e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x46, 0x61, - 0x69, 0x6c, 0x75, 0x72, 0x65, 0x10, 0x07, 0x2a, 0xab, 0x02, 0x0a, 0x0e, 0x43, 0x6f, 0x6d, 0x70, - 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x17, 0x0a, 0x13, 0x55, 0x6e, - 0x64, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x43, 0x6f, 0x6d, 0x70, - 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, 0x12, 0x11, 0x0a, 0x0d, 0x4d, 0x69, 0x78, 0x43, - 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x12, 0x14, 0x0a, 0x10, 0x53, - 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, - 0x04, 0x12, 0x13, 0x0a, 0x0f, 0x4d, 0x69, 0x6e, 0x6f, 0x72, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x13, 0x0a, 0x0f, 0x4d, 0x61, 0x6a, 0x6f, 0x72, 0x43, - 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x06, 0x12, 0x1a, 0x0a, 0x16, 0x4c, - 0x65, 0x76, 0x65, 0x6c, 0x30, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, 0x18, 0x0a, 0x14, 0x43, 0x6c, 0x75, 0x73, 0x74, - 0x65, 0x72, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, - 0x08, 0x12, 0x12, 0x0a, 0x0e, 0x53, 0x6f, 0x72, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x10, 0x09, 0x12, 0x1e, 0x0a, 0x1a, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x6f, 0x72, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x10, 0x0a, 0x12, 0x28, 0x0a, 0x24, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, - 0x69, 0x6e, 0x67, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, - 0x6f, 0x72, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x0b, 0x22, - 0x04, 0x08, 0x01, 0x10, 0x01, 0x2a, 0x60, 0x0a, 0x11, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, - 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56, 0x32, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, - 0x6e, 0x65, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x10, - 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x49, 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x10, - 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, 0x03, 0x12, 0x0d, 0x0a, - 0x09, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, - 0x52, 0x65, 0x74, 0x72, 0x79, 0x10, 0x05, 0x2a, 0x33, 0x0a, 0x12, 0x49, 0x6d, 0x70, 0x6f, 0x72, - 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x32, 0x12, 0x0b, 0x0a, - 0x07, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x4c, 0x30, - 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x01, 0x2a, 0x29, 0x0a, 0x09, - 0x47, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x05, 0x0a, 0x01, 0x5f, 0x10, 0x00, - 0x12, 0x09, 0x0a, 0x05, 0x50, 0x61, 0x75, 0x73, 0x65, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x52, - 0x65, 0x73, 0x75, 0x6d, 0x65, 0x10, 0x02, 0x2a, 0xb2, 0x01, 0x0a, 0x13, 0x43, 0x6f, 0x6d, 0x70, - 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, - 0x0b, 0x0a, 0x07, 0x75, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, - 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x70, - 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x63, - 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x66, 0x61, - 0x69, 0x6c, 0x65, 0x64, 0x10, 0x04, 0x12, 0x0b, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, - 0x74, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x69, 0x6e, 0x67, - 0x10, 0x06, 0x12, 0x0c, 0x0a, 0x08, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x69, 0x6e, 0x67, 0x10, 0x07, - 0x12, 0x0b, 0x0a, 0x07, 0x63, 0x6c, 0x65, 0x61, 0x6e, 0x65, 0x64, 0x10, 0x08, 0x12, 0x0e, 0x0a, - 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x5f, 0x73, 0x61, 0x76, 0x65, 0x64, 0x10, 0x09, 0x12, 0x0d, 0x0a, - 0x09, 0x73, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x10, 0x0a, 0x32, 0x91, 0x2b, 0x0a, - 0x09, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x12, 0x4c, 0x0a, 0x05, 0x46, 0x6c, - 0x75, 0x73, 0x68, 0x12, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x55, 0x0a, 0x08, 0x46, 0x6c, 0x75, 0x73, - 0x68, 0x41, 0x6c, 0x6c, 0x12, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, - 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, - 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x7c, 0x0a, 0x18, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, - 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x33, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0xa5, 0x02, 0x0a, 0x1f, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, + 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x22, 0x0a, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x48, 0x0a, 0x0f, 0x63, 0x75, 0x72, + 0x72, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x0f, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x53, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x65, 0x78, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x65, + 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x53, 0x70, 0x65, 0x63, 0x18, 0x06, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0c, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x53, 0x70, 0x65, 0x63, 0x22, + 0x9a, 0x02, 0x0a, 0x20, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, + 0x61, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x22, 0x0a, 0x0c, 0x6b, 0x65, 0x70, + 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, + 0x0c, 0x6b, 0x65, 0x70, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x48, 0x0a, + 0x0f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0f, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x64, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x32, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4a, 0x6f, 0x62, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, + 0x61, 0x69, 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x22, 0x38, 0x0a, 0x1e, + 0x51, 0x75, 0x65, 0x72, 0x79, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, + 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, + 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x22, 0x37, 0x0a, 0x1d, 0x44, 0x72, 0x6f, 0x70, 0x45, 0x78, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, + 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x2a, + 0x3e, 0x0a, 0x0b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x07, + 0x0a, 0x03, 0x4e, 0x65, 0x77, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x4e, 0x6f, 0x72, 0x6d, 0x61, + 0x6c, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x10, 0x02, + 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x65, 0x64, 0x10, 0x03, 0x2a, + 0x32, 0x0a, 0x0c, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, + 0x0a, 0x0a, 0x06, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x10, 0x00, 0x12, 0x06, 0x0a, 0x02, 0x4c, + 0x30, 0x10, 0x01, 0x12, 0x06, 0x0a, 0x02, 0x4c, 0x31, 0x10, 0x02, 0x12, 0x06, 0x0a, 0x02, 0x4c, + 0x32, 0x10, 0x03, 0x2a, 0x99, 0x01, 0x0a, 0x11, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x57, + 0x61, 0x74, 0x63, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x55, 0x6e, 0x63, + 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x43, 0x6f, 0x6d, + 0x70, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x54, 0x6f, 0x57, 0x61, 0x74, + 0x63, 0x68, 0x10, 0x02, 0x12, 0x10, 0x0a, 0x0c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x53, 0x75, 0x63, + 0x63, 0x65, 0x73, 0x73, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x46, + 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x6f, 0x52, 0x65, + 0x6c, 0x65, 0x61, 0x73, 0x65, 0x10, 0x05, 0x12, 0x12, 0x0a, 0x0e, 0x52, 0x65, 0x6c, 0x65, 0x61, + 0x73, 0x65, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x10, 0x06, 0x12, 0x12, 0x0a, 0x0e, 0x52, + 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x10, 0x07, 0x2a, + 0xab, 0x02, 0x0a, 0x0e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, + 0x70, 0x65, 0x12, 0x17, 0x0a, 0x13, 0x55, 0x6e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x4d, + 0x65, 0x72, 0x67, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, + 0x12, 0x11, 0x0a, 0x0d, 0x4d, 0x69, 0x78, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x10, 0x03, 0x12, 0x14, 0x0a, 0x10, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x43, 0x6f, 0x6d, + 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x04, 0x12, 0x13, 0x0a, 0x0f, 0x4d, 0x69, 0x6e, + 0x6f, 0x72, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x13, + 0x0a, 0x0f, 0x4d, 0x61, 0x6a, 0x6f, 0x72, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x10, 0x06, 0x12, 0x1a, 0x0a, 0x16, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x30, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, + 0x18, 0x0a, 0x14, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6d, + 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x12, 0x12, 0x0a, 0x0e, 0x53, 0x6f, 0x72, + 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x09, 0x12, 0x1e, 0x0a, + 0x1a, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x6f, 0x72, + 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x0a, 0x12, 0x28, 0x0a, + 0x24, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x53, 0x6f, 0x72, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x0b, 0x22, 0x04, 0x08, 0x01, 0x10, 0x01, 0x2a, 0x60, 0x0a, + 0x11, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x56, 0x32, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, + 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x49, 0x6e, 0x50, + 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x61, 0x69, + 0x6c, 0x65, 0x64, 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x52, 0x65, 0x74, 0x72, 0x79, 0x10, 0x05, 0x2a, + 0x33, 0x0a, 0x12, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x56, 0x32, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x4c, 0x30, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x10, 0x01, 0x2a, 0x29, 0x0a, 0x09, 0x47, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, + 0x64, 0x12, 0x05, 0x0a, 0x01, 0x5f, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x50, 0x61, 0x75, 0x73, + 0x65, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x10, 0x02, 0x2a, + 0xb2, 0x01, 0x0a, 0x13, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, + 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x75, 0x6e, 0x6b, 0x6e, 0x6f, + 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6e, + 0x67, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x69, 0x6e, + 0x67, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, + 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, 0x04, 0x12, 0x0b, + 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x61, + 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x12, 0x0c, 0x0a, 0x08, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x69, 0x6e, 0x67, 0x10, 0x07, 0x12, 0x0b, 0x0a, 0x07, 0x63, 0x6c, 0x65, 0x61, + 0x6e, 0x65, 0x64, 0x10, 0x08, 0x12, 0x0e, 0x0a, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x5f, 0x73, 0x61, + 0x76, 0x65, 0x64, 0x10, 0x09, 0x12, 0x0d, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, + 0x69, 0x63, 0x10, 0x0a, 0x32, 0x91, 0x2b, 0x0a, 0x09, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6f, + 0x72, 0x64, 0x12, 0x4c, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x12, 0x1f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x46, 0x6c, 0x75, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x55, 0x0a, 0x08, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x12, 0x22, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7c, 0x0a, 0x18, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x61, 0x0a, - 0x0c, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x26, 0x2e, + 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x33, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x61, 0x0a, 0x0c, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x6d, 0x0a, 0x0f, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x49, 0x44, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x49, 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x03, 0x88, 0x02, 0x01, 0x12, - 0x67, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, - 0x6f, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x0f, 0x41, 0x73, 0x73, 0x69, + 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2a, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x79, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x49, 0x6e, - 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, - 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, 0x6e, + 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, + 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x03, 0x88, 0x02, 0x01, 0x12, 0x67, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, + 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x6d, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x79, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, 0x6e, 0x6c, + 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x49, + 0x6e, 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x49, + 0x6e, 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x82, 0x01, 0x0a, 0x17, 0x47, + 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, + 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, + 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, + 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, + 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x7f, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, + 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, + 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, + 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x6f, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, + 0x66, 0x6f, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, + 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x5b, 0x0a, 0x0f, 0x53, 0x61, 0x76, 0x65, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, + 0x61, 0x74, 0x68, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x61, 0x76, 0x65, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, 0x6e, - 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x82, 0x01, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x31, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7f, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, - 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, - 0x6c, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x49, 0x6e, 0x66, 0x6f, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x0f, 0x53, 0x61, 0x76, - 0x65, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x29, 0x2e, 0x6d, + 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, 0x6a, + 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, + 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x53, 0x61, 0x76, 0x65, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, + 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x70, 0x0a, 0x11, 0x47, 0x65, + 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x56, 0x32, 0x12, + 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x56, 0x32, 0x1a, 0x2c, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x56, 0x32, 0x22, 0x00, 0x12, 0x7f, 0x0a, 0x16, + 0x47, 0x65, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, + 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, + 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, + 0x12, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x73, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, + 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x76, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x42, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, + 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x10, 0x47, 0x65, + 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2c, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7b, 0x0a, + 0x12, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, + 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x47, 0x65, + 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, + 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x10, 0x4d, + 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x70, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x77, + 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, + 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, + 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x80, 0x01, 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x57, 0x69, + 0x74, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, + 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, + 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x67, 0x0a, 0x0d, 0x57, 0x61, + 0x74, 0x63, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x57, 0x61, 0x74, 0x63, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x61, 0x74, 0x63, 0x68, 0x43, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x03, + 0x88, 0x02, 0x01, 0x12, 0x66, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, + 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, 0x12, 0x44, + 0x72, 0x6f, 0x70, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, + 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x6a, 0x0a, 0x0f, 0x53, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6b, 0x0a, 0x17, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, + 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, + 0x69, 0x63, 0x73, 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, 0x6b, 0x0a, 0x17, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, + 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 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, 0x6a, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, - 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, - 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, - 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x70, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, - 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x56, 0x32, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x52, - 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x56, 0x32, 0x1a, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, - 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x56, 0x32, 0x22, 0x00, 0x12, 0x7f, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x30, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x63, - 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, - 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, - 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2c, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, - 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x76, 0x0a, 0x13, 0x47, 0x65, - 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, - 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x73, 0x42, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, - 0x42, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x71, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, - 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, - 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, - 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x12, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, - 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, - 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, - 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, - 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x10, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x43, 0x6f, 0x6d, - 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4d, 0x61, - 0x6e, 0x75, 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4d, 0x61, 0x6e, 0x75, - 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x77, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, - 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2e, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x80, 0x01, 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x57, 0x69, 0x74, 0x68, 0x50, 0x6c, 0x61, 0x6e, 0x73, 0x12, - 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x67, 0x0a, 0x0d, 0x57, 0x61, 0x74, 0x63, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x61, 0x74, 0x63, 0x68, 0x43, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x57, 0x61, 0x74, 0x63, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x03, 0x88, 0x02, 0x01, 0x12, 0x66, 0x0a, 0x0d, 0x47, - 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x27, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x46, - 0x6c, 0x75, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, 0x12, 0x44, 0x72, 0x6f, 0x70, 0x56, 0x69, 0x72, 0x74, 0x75, - 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, - 0x6f, 0x70, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, 0x6f, 0x70, - 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6a, 0x0a, 0x0f, 0x53, 0x65, 0x74, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x29, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x53, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x53, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x12, 0x6b, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, - 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 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, 0x6b, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x31, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x68, - 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 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, 0x63, - 0x0a, 0x13, 0x4d, 0x61, 0x72, 0x6b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x44, 0x72, - 0x6f, 0x70, 0x70, 0x65, 0x64, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x61, 0x72, 0x6b, 0x53, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x52, 0x65, 0x71, + 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x13, 0x4d, 0x61, 0x72, 0x6b, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x12, 0x2d, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x4d, 0x61, 0x72, 0x6b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x44, 0x72, + 0x6f, 0x70, 0x70, 0x65, 0x64, 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, 0x1a, 0x42, + 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x65, 0x64, 0x43, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x6c, + 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 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, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, - 0x41, 0x6c, 0x74, 0x65, 0x72, 0x65, 0x64, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 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, 0x62, 0x0a, 0x0b, 0x43, - 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, - 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x54, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x26, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, - 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, - 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0a, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, - 0x64, 0x65, 0x78, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, - 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, - 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x47, 0x65, 0x74, - 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, - 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, - 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x7b, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, - 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, - 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, - 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, - 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, - 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x64, - 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, - 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, - 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, - 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, - 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 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, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, - 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, - 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x30, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, - 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, - 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, - 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x64, - 0x65, 0x78, 0x65, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, - 0x64, 0x65, 0x78, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x58, 0x0a, 0x09, 0x47, 0x63, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x72, 0x6d, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x72, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x63, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x72, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x65, 0x0a, 0x14, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x44, 0x61, 0x74, 0x61, 0x4e, - 0x6f, 0x64, 0x65, 0x54, 0x74, 0x4d, 0x73, 0x67, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, - 0x70, 0x6f, 0x72, 0x74, 0x44, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x54, 0x74, 0x4d, 0x73, - 0x67, 0x73, 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, 0x4f, 0x0a, 0x09, 0x47, 0x63, 0x43, 0x6f, - 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x74, - 0x72, 0x6f, 0x6c, 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, 0x61, 0x0a, 0x08, 0x49, 0x6d, 0x70, - 0x6f, 0x72, 0x74, 0x56, 0x32, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, - 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, - 0x6e, 0x61, 0x6c, 0x1a, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, - 0x72, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x78, 0x0a, 0x11, - 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, - 0x73, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, - 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 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, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6d, - 0x70, 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6e, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, - 0x70, 0x6f, 0x72, 0x74, 0x73, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x4c, 0x69, - 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, - 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x0f, 0x41, 0x64, 0x64, 0x46, 0x69, 0x6c, - 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x41, 0x64, 0x64, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 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, 0x63, 0x0a, 0x12, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x46, - 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x2e, 0x2e, 0x6d, 0x69, + 0x73, 0x22, 0x00, 0x12, 0x62, 0x0a, 0x0b, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, + 0x74, 0x68, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, + 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 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, 0x74, 0x0a, 0x11, 0x4c, 0x69, - 0x73, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x12, - 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x32, 0xa2, 0x10, 0x0a, 0x08, 0x44, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x6c, 0x0a, - 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, - 0x74, 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, - 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, - 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x14, 0x47, - 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, - 0x6e, 0x65, 0x6c, 0x12, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x53, - 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x74, - 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5b, - 0x0a, 0x0f, 0x57, 0x61, 0x74, 0x63, 0x68, 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, - 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x61, 0x74, 0x63, 0x68, 0x44, 0x6d, 0x43, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x73, 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, 0x57, 0x0a, 0x0d, 0x46, - 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x27, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 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, 0x7b, 0x0a, 0x12, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, - 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, - 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, - 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, - 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x50, 0x0a, 0x0c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x56, 0x32, 0x12, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x6c, 0x61, 0x6e, 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, 0x6d, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, - 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x55, 0x0a, 0x0c, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x73, 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, 0x73, 0x0a, 0x12, 0x52, 0x65, - 0x73, 0x65, 0x6e, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, - 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x6e, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x6e, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x57, 0x0a, 0x0d, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, - 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, - 0x6c, 0x73, 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, 0x64, 0x0a, 0x16, 0x4e, 0x6f, 0x74, 0x69, - 0x66, 0x79, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, - 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x52, 0x0a, + 0x0a, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x25, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, + 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x14, 0x47, 0x65, 0x74, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 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, 0x7b, - 0x0a, 0x1d, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, - 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, - 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x57, 0x61, 0x74, 0x63, 0x68, - 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x33, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, - 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4f, 0x0a, 0x09, 0x50, - 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x72, 0x65, - 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 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, 0x4b, 0x0a, 0x08, - 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x56, 0x32, 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, - 0x6f, 0x72, 0x74, 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, 0x67, 0x0a, 0x0e, 0x51, 0x75, 0x65, - 0x72, 0x79, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x28, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, - 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x0b, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, 0x72, - 0x74, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, 0x72, - 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, - 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0a, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, - 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 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, 0x58, 0x0a, 0x09, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, - 0x6f, 0x74, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, - 0x79, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x61, 0x0a, 0x12, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, - 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x66, + 0x0a, 0x0d, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, + 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, + 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x75, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x2d, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, + 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, + 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, + 0x15, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, + 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, + 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, + 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, + 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x12, 0x26, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x58, 0x0a, 0x09, 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x72, 0x6d, 0x12, 0x23, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x72, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x72, 0x6d, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x65, 0x0a, 0x14, 0x52, 0x65, 0x70, + 0x6f, 0x72, 0x74, 0x44, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x54, 0x74, 0x4d, 0x73, 0x67, + 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x44, 0x61, 0x74, 0x61, + 0x4e, 0x6f, 0x64, 0x65, 0x54, 0x74, 0x4d, 0x73, 0x67, 0x73, 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, 0x4f, 0x0a, 0x09, 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x12, 0x23, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 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, 0x61, 0x0a, 0x08, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x56, 0x32, 0x12, 0x2c, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x1a, 0x25, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, + 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x78, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, + 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, + 0x65, 0x73, 0x73, 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, 0x74, 0x65, 0x72, 0x6e, + 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, 0x67, + 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6e, + 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x12, 0x31, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, + 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, + 0x0a, 0x0f, 0x41, 0x64, 0x64, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x41, 0x64, 0x64, 0x46, 0x69, 0x6c, 0x65, 0x52, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 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, 0x63, 0x0a, + 0x12, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, + 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 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, 0x61, 0x0a, 0x10, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, - 0x79, 0x6e, 0x63, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 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, 0x42, 0x31, 0x5a, 0x2f, 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, 0x64, 0x61, 0x74, 0x61, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x22, 0x00, 0x12, 0x74, 0x0a, 0x11, 0x4c, 0x69, 0x73, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, + 0x73, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, + 0x74, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0xa2, 0x10, 0x0a, 0x08, 0x44, 0x61, 0x74, + 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, + 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x73, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, + 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x32, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, + 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, + 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x0f, 0x57, 0x61, 0x74, 0x63, 0x68, 0x44, + 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x61, + 0x74, 0x63, 0x68, 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 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, 0x57, 0x0a, 0x0d, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 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, 0x7b, 0x0a, 0x12, + 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, + 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, + 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, + 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x50, 0x0a, 0x0c, 0x43, 0x6f, + 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x12, 0x21, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 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, 0x6d, 0x0a, 0x12, + 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x55, 0x0a, 0x0c, 0x53, + 0x79, 0x6e, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x53, 0x79, 0x6e, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 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, 0x73, 0x0a, 0x12, 0x52, 0x65, 0x73, 0x65, 0x6e, 0x64, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, + 0x65, 0x6e, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x6e, + 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0d, 0x46, 0x6c, 0x75, 0x73, 0x68, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, + 0x73, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 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, 0x64, 0x0a, 0x16, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x79, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 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, 0x7b, 0x0a, 0x1d, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, + 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x33, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x4f, 0x0a, 0x09, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 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, 0x4b, 0x0a, 0x08, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x56, 0x32, + 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 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, 0x67, 0x0a, 0x0e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, + 0x6f, 0x72, 0x74, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x72, 0x65, + 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x0b, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0a, 0x44, 0x72, + 0x6f, 0x70, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, 0x6f, + 0x70, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 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, 0x58, 0x0a, + 0x09, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, + 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x61, 0x0a, 0x12, 0x44, 0x72, 0x6f, 0x70, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x2c, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x50, 0x6c, 0x61, 0x6e, 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, 0x61, 0x0a, 0x10, 0x53, 0x79, + 0x6e, 0x63, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x2e, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x46, 0x69, 0x6c, 0x65, 0x52, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 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, 0x42, 0x31, 0x5a, + 0x2f, 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, 0x64, 0x61, 0x74, 0x61, 0x70, 0x62, + 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -11569,7 +11873,7 @@ func file_data_coord_proto_rawDescGZIP() []byte { } var file_data_coord_proto_enumTypes = make([]protoimpl.EnumInfo, 8) -var file_data_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 122) +var file_data_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 126) var file_data_coord_proto_goTypes = []interface{}{ (SegmentType)(0), // 0: milvus.proto.data.SegmentType (SegmentLevel)(0), // 1: milvus.proto.data.SegmentLevel @@ -11688,148 +11992,153 @@ var file_data_coord_proto_goTypes = []interface{}{ (*DropCompactionPlanRequest)(nil), // 114: milvus.proto.data.DropCompactionPlanRequest (*FileResourceInfo)(nil), // 115: milvus.proto.data.FileResourceInfo (*CreateExternalCollectionResponse)(nil), // 116: milvus.proto.data.CreateExternalCollectionResponse - nil, // 117: milvus.proto.data.FlushResponse.ChannelCpsEntry - nil, // 118: milvus.proto.data.FlushResult.ChannelCpsEntry - nil, // 119: milvus.proto.data.FlushAllResponse.FlushAllMsgsEntry - nil, // 120: milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry - nil, // 121: milvus.proto.data.VchannelInfo.PartitionStatsVersionsEntry - nil, // 122: milvus.proto.data.SegmentInfo.TextStatsLogsEntry - nil, // 123: milvus.proto.data.SegmentInfo.JsonKeyStatsEntry - nil, // 124: milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry - nil, // 125: milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry - nil, // 126: milvus.proto.data.CompactionSegment.TextStatsLogsEntry - nil, // 127: milvus.proto.data.PartitionImportStats.PartitionRowsEntry - nil, // 128: milvus.proto.data.PartitionImportStats.PartitionDataSizeEntry - nil, // 129: milvus.proto.data.ImportFileStats.HashedStatsEntry - (*commonpb.MsgBase)(nil), // 130: milvus.proto.common.MsgBase - (*commonpb.Status)(nil), // 131: milvus.proto.common.Status - (*milvuspb.ClusterInfo)(nil), // 132: milvus.proto.milvus.ClusterInfo - (commonpb.SegmentState)(0), // 133: milvus.proto.common.SegmentState - (*msgpb.MsgPosition)(nil), // 134: milvus.proto.msg.MsgPosition - (*internalpb.StringList)(nil), // 135: milvus.proto.internal.StringList - (*commonpb.KeyValuePair)(nil), // 136: milvus.proto.common.KeyValuePair - (*schemapb.CollectionSchema)(nil), // 137: milvus.proto.schema.CollectionSchema - (*commonpb.KeyDataPair)(nil), // 138: milvus.proto.common.KeyDataPair - (*commonpb.SegmentStats)(nil), // 139: milvus.proto.common.SegmentStats - (*schemapb.FieldSchema)(nil), // 140: milvus.proto.schema.FieldSchema - (*msgpb.DataNodeTtMsg)(nil), // 141: milvus.proto.msg.DataNodeTtMsg - (*internalpb.ImportFile)(nil), // 142: milvus.proto.internal.ImportFile - (*indexpb.StorageConfig)(nil), // 143: milvus.proto.index.StorageConfig - (internalpb.ImportJobState)(0), // 144: milvus.proto.internal.ImportJobState - (*commonpb.ImmutableMessage)(nil), // 145: milvus.proto.common.ImmutableMessage - (*msgpb.CreateCollectionRequest)(nil), // 146: milvus.proto.msg.CreateCollectionRequest - (*milvuspb.GetFlushAllStateRequest)(nil), // 147: milvus.proto.milvus.GetFlushAllStateRequest - (*internalpb.ShowConfigurationsRequest)(nil), // 148: milvus.proto.internal.ShowConfigurationsRequest - (*milvuspb.GetMetricsRequest)(nil), // 149: milvus.proto.milvus.GetMetricsRequest - (*milvuspb.ManualCompactionRequest)(nil), // 150: milvus.proto.milvus.ManualCompactionRequest - (*milvuspb.GetCompactionStateRequest)(nil), // 151: milvus.proto.milvus.GetCompactionStateRequest - (*milvuspb.GetCompactionPlansRequest)(nil), // 152: milvus.proto.milvus.GetCompactionPlansRequest - (*milvuspb.CheckHealthRequest)(nil), // 153: milvus.proto.milvus.CheckHealthRequest - (*indexpb.CreateIndexRequest)(nil), // 154: milvus.proto.index.CreateIndexRequest - (*indexpb.AlterIndexRequest)(nil), // 155: milvus.proto.index.AlterIndexRequest - (*indexpb.GetIndexStateRequest)(nil), // 156: milvus.proto.index.GetIndexStateRequest - (*indexpb.GetSegmentIndexStateRequest)(nil), // 157: milvus.proto.index.GetSegmentIndexStateRequest - (*indexpb.GetIndexInfoRequest)(nil), // 158: milvus.proto.index.GetIndexInfoRequest - (*indexpb.DropIndexRequest)(nil), // 159: milvus.proto.index.DropIndexRequest - (*indexpb.DescribeIndexRequest)(nil), // 160: milvus.proto.index.DescribeIndexRequest - (*indexpb.GetIndexStatisticsRequest)(nil), // 161: milvus.proto.index.GetIndexStatisticsRequest - (*indexpb.GetIndexBuildProgressRequest)(nil), // 162: milvus.proto.index.GetIndexBuildProgressRequest - (*indexpb.ListIndexesRequest)(nil), // 163: milvus.proto.index.ListIndexesRequest - (*internalpb.ImportRequestInternal)(nil), // 164: milvus.proto.internal.ImportRequestInternal - (*internalpb.GetImportProgressRequest)(nil), // 165: milvus.proto.internal.GetImportProgressRequest - (*internalpb.ListImportsRequestInternal)(nil), // 166: milvus.proto.internal.ListImportsRequestInternal - (*milvuspb.AddFileResourceRequest)(nil), // 167: milvus.proto.milvus.AddFileResourceRequest - (*milvuspb.RemoveFileResourceRequest)(nil), // 168: milvus.proto.milvus.RemoveFileResourceRequest - (*milvuspb.ListFileResourcesRequest)(nil), // 169: milvus.proto.milvus.ListFileResourcesRequest - (*milvuspb.GetComponentStatesRequest)(nil), // 170: milvus.proto.milvus.GetComponentStatesRequest - (*internalpb.GetStatisticsChannelRequest)(nil), // 171: milvus.proto.internal.GetStatisticsChannelRequest - (*internalpb.SyncFileResourceRequest)(nil), // 172: milvus.proto.internal.SyncFileResourceRequest - (*milvuspb.StringResponse)(nil), // 173: milvus.proto.milvus.StringResponse - (*milvuspb.GetFlushAllStateResponse)(nil), // 174: milvus.proto.milvus.GetFlushAllStateResponse - (*internalpb.ShowConfigurationsResponse)(nil), // 175: milvus.proto.internal.ShowConfigurationsResponse - (*milvuspb.GetMetricsResponse)(nil), // 176: milvus.proto.milvus.GetMetricsResponse - (*milvuspb.ManualCompactionResponse)(nil), // 177: milvus.proto.milvus.ManualCompactionResponse - (*milvuspb.GetCompactionStateResponse)(nil), // 178: milvus.proto.milvus.GetCompactionStateResponse - (*milvuspb.GetCompactionPlansResponse)(nil), // 179: milvus.proto.milvus.GetCompactionPlansResponse - (*milvuspb.GetFlushStateResponse)(nil), // 180: milvus.proto.milvus.GetFlushStateResponse - (*milvuspb.CheckHealthResponse)(nil), // 181: milvus.proto.milvus.CheckHealthResponse - (*indexpb.GetIndexStateResponse)(nil), // 182: milvus.proto.index.GetIndexStateResponse - (*indexpb.GetSegmentIndexStateResponse)(nil), // 183: milvus.proto.index.GetSegmentIndexStateResponse - (*indexpb.GetIndexInfoResponse)(nil), // 184: milvus.proto.index.GetIndexInfoResponse - (*indexpb.DescribeIndexResponse)(nil), // 185: milvus.proto.index.DescribeIndexResponse - (*indexpb.GetIndexStatisticsResponse)(nil), // 186: milvus.proto.index.GetIndexStatisticsResponse - (*indexpb.GetIndexBuildProgressResponse)(nil), // 187: milvus.proto.index.GetIndexBuildProgressResponse - (*indexpb.ListIndexesResponse)(nil), // 188: milvus.proto.index.ListIndexesResponse - (*internalpb.ImportResponse)(nil), // 189: milvus.proto.internal.ImportResponse - (*internalpb.GetImportProgressResponse)(nil), // 190: milvus.proto.internal.GetImportProgressResponse - (*internalpb.ListImportsResponse)(nil), // 191: milvus.proto.internal.ListImportsResponse - (*milvuspb.ListFileResourcesResponse)(nil), // 192: milvus.proto.milvus.ListFileResourcesResponse - (*milvuspb.ComponentStates)(nil), // 193: milvus.proto.milvus.ComponentStates + (*UpdateExternalCollectionRequest)(nil), // 117: milvus.proto.data.UpdateExternalCollectionRequest + (*UpdateExternalCollectionResponse)(nil), // 118: milvus.proto.data.UpdateExternalCollectionResponse + (*QueryExternalCollectionRequest)(nil), // 119: milvus.proto.data.QueryExternalCollectionRequest + (*DropExternalCollectionRequest)(nil), // 120: milvus.proto.data.DropExternalCollectionRequest + nil, // 121: milvus.proto.data.FlushResponse.ChannelCpsEntry + nil, // 122: milvus.proto.data.FlushResult.ChannelCpsEntry + nil, // 123: milvus.proto.data.FlushAllResponse.FlushAllMsgsEntry + nil, // 124: milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry + nil, // 125: milvus.proto.data.VchannelInfo.PartitionStatsVersionsEntry + nil, // 126: milvus.proto.data.SegmentInfo.TextStatsLogsEntry + nil, // 127: milvus.proto.data.SegmentInfo.JsonKeyStatsEntry + nil, // 128: milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry + nil, // 129: milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry + nil, // 130: milvus.proto.data.CompactionSegment.TextStatsLogsEntry + nil, // 131: milvus.proto.data.PartitionImportStats.PartitionRowsEntry + nil, // 132: milvus.proto.data.PartitionImportStats.PartitionDataSizeEntry + nil, // 133: milvus.proto.data.ImportFileStats.HashedStatsEntry + (*commonpb.MsgBase)(nil), // 134: milvus.proto.common.MsgBase + (*commonpb.Status)(nil), // 135: milvus.proto.common.Status + (*milvuspb.ClusterInfo)(nil), // 136: milvus.proto.milvus.ClusterInfo + (commonpb.SegmentState)(0), // 137: milvus.proto.common.SegmentState + (*msgpb.MsgPosition)(nil), // 138: milvus.proto.msg.MsgPosition + (*internalpb.StringList)(nil), // 139: milvus.proto.internal.StringList + (*commonpb.KeyValuePair)(nil), // 140: milvus.proto.common.KeyValuePair + (*schemapb.CollectionSchema)(nil), // 141: milvus.proto.schema.CollectionSchema + (*commonpb.KeyDataPair)(nil), // 142: milvus.proto.common.KeyDataPair + (*commonpb.SegmentStats)(nil), // 143: milvus.proto.common.SegmentStats + (*schemapb.FieldSchema)(nil), // 144: milvus.proto.schema.FieldSchema + (*msgpb.DataNodeTtMsg)(nil), // 145: milvus.proto.msg.DataNodeTtMsg + (*internalpb.ImportFile)(nil), // 146: milvus.proto.internal.ImportFile + (*indexpb.StorageConfig)(nil), // 147: milvus.proto.index.StorageConfig + (internalpb.ImportJobState)(0), // 148: milvus.proto.internal.ImportJobState + (indexpb.JobState)(0), // 149: milvus.proto.index.JobState + (*commonpb.ImmutableMessage)(nil), // 150: milvus.proto.common.ImmutableMessage + (*msgpb.CreateCollectionRequest)(nil), // 151: milvus.proto.msg.CreateCollectionRequest + (*milvuspb.GetFlushAllStateRequest)(nil), // 152: milvus.proto.milvus.GetFlushAllStateRequest + (*internalpb.ShowConfigurationsRequest)(nil), // 153: milvus.proto.internal.ShowConfigurationsRequest + (*milvuspb.GetMetricsRequest)(nil), // 154: milvus.proto.milvus.GetMetricsRequest + (*milvuspb.ManualCompactionRequest)(nil), // 155: milvus.proto.milvus.ManualCompactionRequest + (*milvuspb.GetCompactionStateRequest)(nil), // 156: milvus.proto.milvus.GetCompactionStateRequest + (*milvuspb.GetCompactionPlansRequest)(nil), // 157: milvus.proto.milvus.GetCompactionPlansRequest + (*milvuspb.CheckHealthRequest)(nil), // 158: milvus.proto.milvus.CheckHealthRequest + (*indexpb.CreateIndexRequest)(nil), // 159: milvus.proto.index.CreateIndexRequest + (*indexpb.AlterIndexRequest)(nil), // 160: milvus.proto.index.AlterIndexRequest + (*indexpb.GetIndexStateRequest)(nil), // 161: milvus.proto.index.GetIndexStateRequest + (*indexpb.GetSegmentIndexStateRequest)(nil), // 162: milvus.proto.index.GetSegmentIndexStateRequest + (*indexpb.GetIndexInfoRequest)(nil), // 163: milvus.proto.index.GetIndexInfoRequest + (*indexpb.DropIndexRequest)(nil), // 164: milvus.proto.index.DropIndexRequest + (*indexpb.DescribeIndexRequest)(nil), // 165: milvus.proto.index.DescribeIndexRequest + (*indexpb.GetIndexStatisticsRequest)(nil), // 166: milvus.proto.index.GetIndexStatisticsRequest + (*indexpb.GetIndexBuildProgressRequest)(nil), // 167: milvus.proto.index.GetIndexBuildProgressRequest + (*indexpb.ListIndexesRequest)(nil), // 168: milvus.proto.index.ListIndexesRequest + (*internalpb.ImportRequestInternal)(nil), // 169: milvus.proto.internal.ImportRequestInternal + (*internalpb.GetImportProgressRequest)(nil), // 170: milvus.proto.internal.GetImportProgressRequest + (*internalpb.ListImportsRequestInternal)(nil), // 171: milvus.proto.internal.ListImportsRequestInternal + (*milvuspb.AddFileResourceRequest)(nil), // 172: milvus.proto.milvus.AddFileResourceRequest + (*milvuspb.RemoveFileResourceRequest)(nil), // 173: milvus.proto.milvus.RemoveFileResourceRequest + (*milvuspb.ListFileResourcesRequest)(nil), // 174: milvus.proto.milvus.ListFileResourcesRequest + (*milvuspb.GetComponentStatesRequest)(nil), // 175: milvus.proto.milvus.GetComponentStatesRequest + (*internalpb.GetStatisticsChannelRequest)(nil), // 176: milvus.proto.internal.GetStatisticsChannelRequest + (*internalpb.SyncFileResourceRequest)(nil), // 177: milvus.proto.internal.SyncFileResourceRequest + (*milvuspb.StringResponse)(nil), // 178: milvus.proto.milvus.StringResponse + (*milvuspb.GetFlushAllStateResponse)(nil), // 179: milvus.proto.milvus.GetFlushAllStateResponse + (*internalpb.ShowConfigurationsResponse)(nil), // 180: milvus.proto.internal.ShowConfigurationsResponse + (*milvuspb.GetMetricsResponse)(nil), // 181: milvus.proto.milvus.GetMetricsResponse + (*milvuspb.ManualCompactionResponse)(nil), // 182: milvus.proto.milvus.ManualCompactionResponse + (*milvuspb.GetCompactionStateResponse)(nil), // 183: milvus.proto.milvus.GetCompactionStateResponse + (*milvuspb.GetCompactionPlansResponse)(nil), // 184: milvus.proto.milvus.GetCompactionPlansResponse + (*milvuspb.GetFlushStateResponse)(nil), // 185: milvus.proto.milvus.GetFlushStateResponse + (*milvuspb.CheckHealthResponse)(nil), // 186: milvus.proto.milvus.CheckHealthResponse + (*indexpb.GetIndexStateResponse)(nil), // 187: milvus.proto.index.GetIndexStateResponse + (*indexpb.GetSegmentIndexStateResponse)(nil), // 188: milvus.proto.index.GetSegmentIndexStateResponse + (*indexpb.GetIndexInfoResponse)(nil), // 189: milvus.proto.index.GetIndexInfoResponse + (*indexpb.DescribeIndexResponse)(nil), // 190: milvus.proto.index.DescribeIndexResponse + (*indexpb.GetIndexStatisticsResponse)(nil), // 191: milvus.proto.index.GetIndexStatisticsResponse + (*indexpb.GetIndexBuildProgressResponse)(nil), // 192: milvus.proto.index.GetIndexBuildProgressResponse + (*indexpb.ListIndexesResponse)(nil), // 193: milvus.proto.index.ListIndexesResponse + (*internalpb.ImportResponse)(nil), // 194: milvus.proto.internal.ImportResponse + (*internalpb.GetImportProgressResponse)(nil), // 195: milvus.proto.internal.GetImportProgressResponse + (*internalpb.ListImportsResponse)(nil), // 196: milvus.proto.internal.ListImportsResponse + (*milvuspb.ListFileResourcesResponse)(nil), // 197: milvus.proto.milvus.ListFileResourcesResponse + (*milvuspb.ComponentStates)(nil), // 198: milvus.proto.milvus.ComponentStates } var file_data_coord_proto_depIdxs = []int32{ - 130, // 0: milvus.proto.data.FlushRequest.base:type_name -> milvus.proto.common.MsgBase - 131, // 1: milvus.proto.data.FlushResponse.status:type_name -> milvus.proto.common.Status - 117, // 2: milvus.proto.data.FlushResponse.channel_cps:type_name -> milvus.proto.data.FlushResponse.ChannelCpsEntry - 118, // 3: milvus.proto.data.FlushResult.channel_cps:type_name -> milvus.proto.data.FlushResult.ChannelCpsEntry - 130, // 4: milvus.proto.data.FlushAllRequest.base:type_name -> milvus.proto.common.MsgBase + 134, // 0: milvus.proto.data.FlushRequest.base:type_name -> milvus.proto.common.MsgBase + 135, // 1: milvus.proto.data.FlushResponse.status:type_name -> milvus.proto.common.Status + 121, // 2: milvus.proto.data.FlushResponse.channel_cps:type_name -> milvus.proto.data.FlushResponse.ChannelCpsEntry + 122, // 3: milvus.proto.data.FlushResult.channel_cps:type_name -> milvus.proto.data.FlushResult.ChannelCpsEntry + 134, // 4: milvus.proto.data.FlushAllRequest.base:type_name -> milvus.proto.common.MsgBase 13, // 5: milvus.proto.data.FlushAllRequest.flush_targets:type_name -> milvus.proto.data.FlushAllTarget - 131, // 6: milvus.proto.data.FlushAllResponse.status:type_name -> milvus.proto.common.Status + 135, // 6: milvus.proto.data.FlushAllResponse.status:type_name -> milvus.proto.common.Status 11, // 7: milvus.proto.data.FlushAllResponse.flush_results:type_name -> milvus.proto.data.FlushResult - 119, // 8: milvus.proto.data.FlushAllResponse.flush_all_msgs:type_name -> milvus.proto.data.FlushAllResponse.FlushAllMsgsEntry - 132, // 9: milvus.proto.data.FlushAllResponse.cluster_info:type_name -> milvus.proto.milvus.ClusterInfo - 130, // 10: milvus.proto.data.FlushChannelsRequest.base:type_name -> milvus.proto.common.MsgBase + 123, // 8: milvus.proto.data.FlushAllResponse.flush_all_msgs:type_name -> milvus.proto.data.FlushAllResponse.FlushAllMsgsEntry + 136, // 9: milvus.proto.data.FlushAllResponse.cluster_info:type_name -> milvus.proto.milvus.ClusterInfo + 134, // 10: milvus.proto.data.FlushChannelsRequest.base:type_name -> milvus.proto.common.MsgBase 1, // 11: milvus.proto.data.SegmentIDRequest.level:type_name -> milvus.proto.data.SegmentLevel 38, // 12: milvus.proto.data.AllocSegmentResponse.segment_info:type_name -> milvus.proto.data.SegmentInfo - 131, // 13: milvus.proto.data.AllocSegmentResponse.status:type_name -> milvus.proto.common.Status + 135, // 13: milvus.proto.data.AllocSegmentResponse.status:type_name -> milvus.proto.common.Status 16, // 14: milvus.proto.data.AssignSegmentIDRequest.segmentIDRequests:type_name -> milvus.proto.data.SegmentIDRequest - 131, // 15: milvus.proto.data.SegmentIDAssignment.status:type_name -> milvus.proto.common.Status + 135, // 15: milvus.proto.data.SegmentIDAssignment.status:type_name -> milvus.proto.common.Status 20, // 16: milvus.proto.data.AssignSegmentIDResponse.segIDAssignments:type_name -> milvus.proto.data.SegmentIDAssignment - 131, // 17: milvus.proto.data.AssignSegmentIDResponse.status:type_name -> milvus.proto.common.Status - 130, // 18: milvus.proto.data.GetSegmentStatesRequest.base:type_name -> milvus.proto.common.MsgBase - 133, // 19: milvus.proto.data.SegmentStateInfo.state:type_name -> milvus.proto.common.SegmentState - 134, // 20: milvus.proto.data.SegmentStateInfo.start_position:type_name -> milvus.proto.msg.MsgPosition - 134, // 21: milvus.proto.data.SegmentStateInfo.end_position:type_name -> milvus.proto.msg.MsgPosition - 131, // 22: milvus.proto.data.SegmentStateInfo.status:type_name -> milvus.proto.common.Status - 131, // 23: milvus.proto.data.GetSegmentStatesResponse.status:type_name -> milvus.proto.common.Status + 135, // 17: milvus.proto.data.AssignSegmentIDResponse.status:type_name -> milvus.proto.common.Status + 134, // 18: milvus.proto.data.GetSegmentStatesRequest.base:type_name -> milvus.proto.common.MsgBase + 137, // 19: milvus.proto.data.SegmentStateInfo.state:type_name -> milvus.proto.common.SegmentState + 138, // 20: milvus.proto.data.SegmentStateInfo.start_position:type_name -> milvus.proto.msg.MsgPosition + 138, // 21: milvus.proto.data.SegmentStateInfo.end_position:type_name -> milvus.proto.msg.MsgPosition + 135, // 22: milvus.proto.data.SegmentStateInfo.status:type_name -> milvus.proto.common.Status + 135, // 23: milvus.proto.data.GetSegmentStatesResponse.status:type_name -> milvus.proto.common.Status 23, // 24: milvus.proto.data.GetSegmentStatesResponse.states:type_name -> milvus.proto.data.SegmentStateInfo - 130, // 25: milvus.proto.data.GetSegmentInfoRequest.base:type_name -> milvus.proto.common.MsgBase - 131, // 26: milvus.proto.data.GetSegmentInfoResponse.status:type_name -> milvus.proto.common.Status + 134, // 25: milvus.proto.data.GetSegmentInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 135, // 26: milvus.proto.data.GetSegmentInfoResponse.status:type_name -> milvus.proto.common.Status 38, // 27: milvus.proto.data.GetSegmentInfoResponse.infos:type_name -> milvus.proto.data.SegmentInfo - 120, // 28: milvus.proto.data.GetSegmentInfoResponse.channel_checkpoint:type_name -> milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry - 130, // 29: milvus.proto.data.GetInsertBinlogPathsRequest.base:type_name -> milvus.proto.common.MsgBase - 135, // 30: milvus.proto.data.GetInsertBinlogPathsResponse.paths:type_name -> milvus.proto.internal.StringList - 131, // 31: milvus.proto.data.GetInsertBinlogPathsResponse.status:type_name -> milvus.proto.common.Status - 130, // 32: milvus.proto.data.GetCollectionStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase - 136, // 33: milvus.proto.data.GetCollectionStatisticsResponse.stats:type_name -> milvus.proto.common.KeyValuePair - 131, // 34: milvus.proto.data.GetCollectionStatisticsResponse.status:type_name -> milvus.proto.common.Status - 130, // 35: milvus.proto.data.GetPartitionStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase - 136, // 36: milvus.proto.data.GetPartitionStatisticsResponse.stats:type_name -> milvus.proto.common.KeyValuePair - 131, // 37: milvus.proto.data.GetPartitionStatisticsResponse.status:type_name -> milvus.proto.common.Status - 134, // 38: milvus.proto.data.VchannelInfo.seek_position:type_name -> milvus.proto.msg.MsgPosition + 124, // 28: milvus.proto.data.GetSegmentInfoResponse.channel_checkpoint:type_name -> milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry + 134, // 29: milvus.proto.data.GetInsertBinlogPathsRequest.base:type_name -> milvus.proto.common.MsgBase + 139, // 30: milvus.proto.data.GetInsertBinlogPathsResponse.paths:type_name -> milvus.proto.internal.StringList + 135, // 31: milvus.proto.data.GetInsertBinlogPathsResponse.status:type_name -> milvus.proto.common.Status + 134, // 32: milvus.proto.data.GetCollectionStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase + 140, // 33: milvus.proto.data.GetCollectionStatisticsResponse.stats:type_name -> milvus.proto.common.KeyValuePair + 135, // 34: milvus.proto.data.GetCollectionStatisticsResponse.status:type_name -> milvus.proto.common.Status + 134, // 35: milvus.proto.data.GetPartitionStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase + 140, // 36: milvus.proto.data.GetPartitionStatisticsResponse.stats:type_name -> milvus.proto.common.KeyValuePair + 135, // 37: milvus.proto.data.GetPartitionStatisticsResponse.status:type_name -> milvus.proto.common.Status + 138, // 38: milvus.proto.data.VchannelInfo.seek_position:type_name -> milvus.proto.msg.MsgPosition 38, // 39: milvus.proto.data.VchannelInfo.unflushedSegments:type_name -> milvus.proto.data.SegmentInfo 38, // 40: milvus.proto.data.VchannelInfo.flushedSegments:type_name -> milvus.proto.data.SegmentInfo 38, // 41: milvus.proto.data.VchannelInfo.dropped_segments:type_name -> milvus.proto.data.SegmentInfo 38, // 42: milvus.proto.data.VchannelInfo.indexed_segments:type_name -> milvus.proto.data.SegmentInfo - 121, // 43: milvus.proto.data.VchannelInfo.partition_stats_versions:type_name -> milvus.proto.data.VchannelInfo.PartitionStatsVersionsEntry - 134, // 44: milvus.proto.data.VchannelInfo.delete_checkpoint:type_name -> milvus.proto.msg.MsgPosition - 130, // 45: milvus.proto.data.WatchDmChannelsRequest.base:type_name -> milvus.proto.common.MsgBase + 125, // 43: milvus.proto.data.VchannelInfo.partition_stats_versions:type_name -> milvus.proto.data.VchannelInfo.PartitionStatsVersionsEntry + 138, // 44: milvus.proto.data.VchannelInfo.delete_checkpoint:type_name -> milvus.proto.msg.MsgPosition + 134, // 45: milvus.proto.data.WatchDmChannelsRequest.base:type_name -> milvus.proto.common.MsgBase 34, // 46: milvus.proto.data.WatchDmChannelsRequest.vchannels:type_name -> milvus.proto.data.VchannelInfo - 130, // 47: milvus.proto.data.FlushSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase - 130, // 48: milvus.proto.data.SegmentMsg.base:type_name -> milvus.proto.common.MsgBase + 134, // 47: milvus.proto.data.FlushSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 134, // 48: milvus.proto.data.SegmentMsg.base:type_name -> milvus.proto.common.MsgBase 38, // 49: milvus.proto.data.SegmentMsg.segment:type_name -> milvus.proto.data.SegmentInfo - 133, // 50: milvus.proto.data.SegmentInfo.state:type_name -> milvus.proto.common.SegmentState - 134, // 51: milvus.proto.data.SegmentInfo.start_position:type_name -> milvus.proto.msg.MsgPosition - 134, // 52: milvus.proto.data.SegmentInfo.dml_position:type_name -> milvus.proto.msg.MsgPosition + 137, // 50: milvus.proto.data.SegmentInfo.state:type_name -> milvus.proto.common.SegmentState + 138, // 51: milvus.proto.data.SegmentInfo.start_position:type_name -> milvus.proto.msg.MsgPosition + 138, // 52: milvus.proto.data.SegmentInfo.dml_position:type_name -> milvus.proto.msg.MsgPosition 46, // 53: milvus.proto.data.SegmentInfo.binlogs:type_name -> milvus.proto.data.FieldBinlog 46, // 54: milvus.proto.data.SegmentInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog 46, // 55: milvus.proto.data.SegmentInfo.deltalogs:type_name -> milvus.proto.data.FieldBinlog 1, // 56: milvus.proto.data.SegmentInfo.level:type_name -> milvus.proto.data.SegmentLevel 1, // 57: milvus.proto.data.SegmentInfo.last_level:type_name -> milvus.proto.data.SegmentLevel - 122, // 58: milvus.proto.data.SegmentInfo.textStatsLogs:type_name -> milvus.proto.data.SegmentInfo.TextStatsLogsEntry + 126, // 58: milvus.proto.data.SegmentInfo.textStatsLogs:type_name -> milvus.proto.data.SegmentInfo.TextStatsLogsEntry 46, // 59: milvus.proto.data.SegmentInfo.bm25statslogs:type_name -> milvus.proto.data.FieldBinlog - 123, // 60: milvus.proto.data.SegmentInfo.jsonKeyStats:type_name -> milvus.proto.data.SegmentInfo.JsonKeyStatsEntry - 134, // 61: milvus.proto.data.SegmentStartPosition.start_position:type_name -> milvus.proto.msg.MsgPosition - 130, // 62: milvus.proto.data.SaveBinlogPathsRequest.base:type_name -> milvus.proto.common.MsgBase + 127, // 60: milvus.proto.data.SegmentInfo.jsonKeyStats:type_name -> milvus.proto.data.SegmentInfo.JsonKeyStatsEntry + 138, // 61: milvus.proto.data.SegmentStartPosition.start_position:type_name -> milvus.proto.msg.MsgPosition + 134, // 62: milvus.proto.data.SaveBinlogPathsRequest.base:type_name -> milvus.proto.common.MsgBase 46, // 63: milvus.proto.data.SaveBinlogPathsRequest.field2BinlogPaths:type_name -> milvus.proto.data.FieldBinlog 41, // 64: milvus.proto.data.SaveBinlogPathsRequest.checkPoints:type_name -> milvus.proto.data.CheckPoint 39, // 65: milvus.proto.data.SaveBinlogPathsRequest.start_positions:type_name -> milvus.proto.data.SegmentStartPosition @@ -11837,305 +12146,310 @@ var file_data_coord_proto_depIdxs = []int32{ 46, // 67: milvus.proto.data.SaveBinlogPathsRequest.deltalogs:type_name -> milvus.proto.data.FieldBinlog 1, // 68: milvus.proto.data.SaveBinlogPathsRequest.seg_level:type_name -> milvus.proto.data.SegmentLevel 46, // 69: milvus.proto.data.SaveBinlogPathsRequest.field2Bm25logPaths:type_name -> milvus.proto.data.FieldBinlog - 134, // 70: milvus.proto.data.CheckPoint.position:type_name -> milvus.proto.msg.MsgPosition + 138, // 70: milvus.proto.data.CheckPoint.position:type_name -> milvus.proto.msg.MsgPosition 2, // 71: milvus.proto.data.ChannelStatus.state:type_name -> milvus.proto.data.ChannelWatchState 43, // 72: milvus.proto.data.DataNodeInfo.channels:type_name -> milvus.proto.data.ChannelStatus 46, // 73: milvus.proto.data.SegmentBinlogs.fieldBinlogs:type_name -> milvus.proto.data.FieldBinlog 46, // 74: milvus.proto.data.SegmentBinlogs.statslogs:type_name -> milvus.proto.data.FieldBinlog 46, // 75: milvus.proto.data.SegmentBinlogs.deltalogs:type_name -> milvus.proto.data.FieldBinlog - 124, // 76: milvus.proto.data.SegmentBinlogs.textStatsLogs:type_name -> milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry + 128, // 76: milvus.proto.data.SegmentBinlogs.textStatsLogs:type_name -> milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry 49, // 77: milvus.proto.data.FieldBinlog.binlogs:type_name -> milvus.proto.data.Binlog - 131, // 78: milvus.proto.data.GetRecoveryInfoResponse.status:type_name -> milvus.proto.common.Status + 135, // 78: milvus.proto.data.GetRecoveryInfoResponse.status:type_name -> milvus.proto.common.Status 34, // 79: milvus.proto.data.GetRecoveryInfoResponse.channels:type_name -> milvus.proto.data.VchannelInfo 45, // 80: milvus.proto.data.GetRecoveryInfoResponse.binlogs:type_name -> milvus.proto.data.SegmentBinlogs - 130, // 81: milvus.proto.data.GetRecoveryInfoRequest.base:type_name -> milvus.proto.common.MsgBase - 131, // 82: milvus.proto.data.GetRecoveryInfoResponseV2.status:type_name -> milvus.proto.common.Status + 134, // 81: milvus.proto.data.GetRecoveryInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 135, // 82: milvus.proto.data.GetRecoveryInfoResponseV2.status:type_name -> milvus.proto.common.Status 34, // 83: milvus.proto.data.GetRecoveryInfoResponseV2.channels:type_name -> milvus.proto.data.VchannelInfo 38, // 84: milvus.proto.data.GetRecoveryInfoResponseV2.segments:type_name -> milvus.proto.data.SegmentInfo - 130, // 85: milvus.proto.data.GetRecoveryInfoRequestV2.base:type_name -> milvus.proto.common.MsgBase - 130, // 86: milvus.proto.data.GetChannelRecoveryInfoRequest.base:type_name -> milvus.proto.common.MsgBase - 131, // 87: milvus.proto.data.GetChannelRecoveryInfoResponse.status:type_name -> milvus.proto.common.Status + 134, // 85: milvus.proto.data.GetRecoveryInfoRequestV2.base:type_name -> milvus.proto.common.MsgBase + 134, // 86: milvus.proto.data.GetChannelRecoveryInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 135, // 87: milvus.proto.data.GetChannelRecoveryInfoResponse.status:type_name -> milvus.proto.common.Status 34, // 88: milvus.proto.data.GetChannelRecoveryInfoResponse.info:type_name -> milvus.proto.data.VchannelInfo - 137, // 89: milvus.proto.data.GetChannelRecoveryInfoResponse.schema:type_name -> milvus.proto.schema.CollectionSchema + 141, // 89: milvus.proto.data.GetChannelRecoveryInfoResponse.schema:type_name -> milvus.proto.schema.CollectionSchema 56, // 90: milvus.proto.data.GetChannelRecoveryInfoResponse.segments_not_created_by_streaming:type_name -> milvus.proto.data.SegmentNotCreatedByStreaming - 130, // 91: milvus.proto.data.GetSegmentsByStatesRequest.base:type_name -> milvus.proto.common.MsgBase - 133, // 92: milvus.proto.data.GetSegmentsByStatesRequest.states:type_name -> milvus.proto.common.SegmentState - 131, // 93: milvus.proto.data.GetSegmentsByStatesResponse.status:type_name -> milvus.proto.common.Status - 130, // 94: milvus.proto.data.GetFlushedSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase - 131, // 95: milvus.proto.data.GetFlushedSegmentsResponse.status:type_name -> milvus.proto.common.Status - 130, // 96: milvus.proto.data.SegmentFlushCompletedMsg.base:type_name -> milvus.proto.common.MsgBase + 134, // 91: milvus.proto.data.GetSegmentsByStatesRequest.base:type_name -> milvus.proto.common.MsgBase + 137, // 92: milvus.proto.data.GetSegmentsByStatesRequest.states:type_name -> milvus.proto.common.SegmentState + 135, // 93: milvus.proto.data.GetSegmentsByStatesResponse.status:type_name -> milvus.proto.common.Status + 134, // 94: milvus.proto.data.GetFlushedSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 135, // 95: milvus.proto.data.GetFlushedSegmentsResponse.status:type_name -> milvus.proto.common.Status + 134, // 96: milvus.proto.data.SegmentFlushCompletedMsg.base:type_name -> milvus.proto.common.MsgBase 38, // 97: milvus.proto.data.SegmentFlushCompletedMsg.segment:type_name -> milvus.proto.data.SegmentInfo 34, // 98: milvus.proto.data.ChannelWatchInfo.vchan:type_name -> milvus.proto.data.VchannelInfo 2, // 99: milvus.proto.data.ChannelWatchInfo.state:type_name -> milvus.proto.data.ChannelWatchState - 137, // 100: milvus.proto.data.ChannelWatchInfo.schema:type_name -> milvus.proto.schema.CollectionSchema - 136, // 101: milvus.proto.data.ChannelWatchInfo.dbProperties:type_name -> milvus.proto.common.KeyValuePair - 130, // 102: milvus.proto.data.CompactionStateRequest.base:type_name -> milvus.proto.common.MsgBase + 141, // 100: milvus.proto.data.ChannelWatchInfo.schema:type_name -> milvus.proto.schema.CollectionSchema + 140, // 101: milvus.proto.data.ChannelWatchInfo.dbProperties:type_name -> milvus.proto.common.KeyValuePair + 134, // 102: milvus.proto.data.CompactionStateRequest.base:type_name -> milvus.proto.common.MsgBase 46, // 103: milvus.proto.data.SyncSegmentInfo.pk_stats_log:type_name -> milvus.proto.data.FieldBinlog - 133, // 104: milvus.proto.data.SyncSegmentInfo.state:type_name -> milvus.proto.common.SegmentState + 137, // 104: milvus.proto.data.SyncSegmentInfo.state:type_name -> milvus.proto.common.SegmentState 1, // 105: milvus.proto.data.SyncSegmentInfo.level:type_name -> milvus.proto.data.SegmentLevel 46, // 106: milvus.proto.data.SyncSegmentsRequest.stats_logs:type_name -> milvus.proto.data.FieldBinlog - 125, // 107: milvus.proto.data.SyncSegmentsRequest.segment_infos:type_name -> milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry + 129, // 107: milvus.proto.data.SyncSegmentsRequest.segment_infos:type_name -> milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry 46, // 108: milvus.proto.data.CompactionSegmentBinlogs.fieldBinlogs:type_name -> milvus.proto.data.FieldBinlog 46, // 109: milvus.proto.data.CompactionSegmentBinlogs.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog 46, // 110: milvus.proto.data.CompactionSegmentBinlogs.deltalogs:type_name -> milvus.proto.data.FieldBinlog 1, // 111: milvus.proto.data.CompactionSegmentBinlogs.level:type_name -> milvus.proto.data.SegmentLevel 66, // 112: milvus.proto.data.CompactionPlan.segmentBinlogs:type_name -> milvus.proto.data.CompactionSegmentBinlogs 3, // 113: milvus.proto.data.CompactionPlan.type:type_name -> milvus.proto.data.CompactionType - 137, // 114: milvus.proto.data.CompactionPlan.schema:type_name -> milvus.proto.schema.CollectionSchema + 141, // 114: milvus.proto.data.CompactionPlan.schema:type_name -> milvus.proto.schema.CollectionSchema 94, // 115: milvus.proto.data.CompactionPlan.pre_allocated_segmentIDs:type_name -> milvus.proto.data.IDRange 94, // 116: milvus.proto.data.CompactionPlan.pre_allocated_logIDs:type_name -> milvus.proto.data.IDRange - 136, // 117: milvus.proto.data.CompactionPlan.plugin_context:type_name -> milvus.proto.common.KeyValuePair + 140, // 117: milvus.proto.data.CompactionPlan.plugin_context:type_name -> milvus.proto.common.KeyValuePair 46, // 118: milvus.proto.data.CompactionSegment.insert_logs:type_name -> milvus.proto.data.FieldBinlog 46, // 119: milvus.proto.data.CompactionSegment.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog 46, // 120: milvus.proto.data.CompactionSegment.deltalogs:type_name -> milvus.proto.data.FieldBinlog 46, // 121: milvus.proto.data.CompactionSegment.bm25logs:type_name -> milvus.proto.data.FieldBinlog - 126, // 122: milvus.proto.data.CompactionSegment.text_stats_logs:type_name -> milvus.proto.data.CompactionSegment.TextStatsLogsEntry + 130, // 122: milvus.proto.data.CompactionSegment.text_stats_logs:type_name -> milvus.proto.data.CompactionSegment.TextStatsLogsEntry 7, // 123: milvus.proto.data.CompactionPlanResult.state:type_name -> milvus.proto.data.CompactionTaskState 68, // 124: milvus.proto.data.CompactionPlanResult.segments:type_name -> milvus.proto.data.CompactionSegment 3, // 125: milvus.proto.data.CompactionPlanResult.type:type_name -> milvus.proto.data.CompactionType - 131, // 126: milvus.proto.data.CompactionStateResponse.status:type_name -> milvus.proto.common.Status + 135, // 126: milvus.proto.data.CompactionStateResponse.status:type_name -> milvus.proto.common.Status 69, // 127: milvus.proto.data.CompactionStateResponse.results:type_name -> milvus.proto.data.CompactionPlanResult - 138, // 128: milvus.proto.data.WatchChannelsRequest.start_positions:type_name -> milvus.proto.common.KeyDataPair - 137, // 129: milvus.proto.data.WatchChannelsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 136, // 130: milvus.proto.data.WatchChannelsRequest.db_properties:type_name -> milvus.proto.common.KeyValuePair - 131, // 131: milvus.proto.data.WatchChannelsResponse.status:type_name -> milvus.proto.common.Status - 130, // 132: milvus.proto.data.SetSegmentStateRequest.base:type_name -> milvus.proto.common.MsgBase - 133, // 133: milvus.proto.data.SetSegmentStateRequest.new_state:type_name -> milvus.proto.common.SegmentState - 131, // 134: milvus.proto.data.SetSegmentStateResponse.status:type_name -> milvus.proto.common.Status - 130, // 135: milvus.proto.data.DropVirtualChannelRequest.base:type_name -> milvus.proto.common.MsgBase + 142, // 128: milvus.proto.data.WatchChannelsRequest.start_positions:type_name -> milvus.proto.common.KeyDataPair + 141, // 129: milvus.proto.data.WatchChannelsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 140, // 130: milvus.proto.data.WatchChannelsRequest.db_properties:type_name -> milvus.proto.common.KeyValuePair + 135, // 131: milvus.proto.data.WatchChannelsResponse.status:type_name -> milvus.proto.common.Status + 134, // 132: milvus.proto.data.SetSegmentStateRequest.base:type_name -> milvus.proto.common.MsgBase + 137, // 133: milvus.proto.data.SetSegmentStateRequest.new_state:type_name -> milvus.proto.common.SegmentState + 135, // 134: milvus.proto.data.SetSegmentStateResponse.status:type_name -> milvus.proto.common.Status + 134, // 135: milvus.proto.data.DropVirtualChannelRequest.base:type_name -> milvus.proto.common.MsgBase 77, // 136: milvus.proto.data.DropVirtualChannelRequest.segments:type_name -> milvus.proto.data.DropVirtualChannelSegment 46, // 137: milvus.proto.data.DropVirtualChannelSegment.field2BinlogPaths:type_name -> milvus.proto.data.FieldBinlog 46, // 138: milvus.proto.data.DropVirtualChannelSegment.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog 46, // 139: milvus.proto.data.DropVirtualChannelSegment.deltalogs:type_name -> milvus.proto.data.FieldBinlog - 134, // 140: milvus.proto.data.DropVirtualChannelSegment.startPosition:type_name -> milvus.proto.msg.MsgPosition - 134, // 141: milvus.proto.data.DropVirtualChannelSegment.checkPoint:type_name -> milvus.proto.msg.MsgPosition - 131, // 142: milvus.proto.data.DropVirtualChannelResponse.status:type_name -> milvus.proto.common.Status - 130, // 143: milvus.proto.data.UpdateSegmentStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase - 139, // 144: milvus.proto.data.UpdateSegmentStatisticsRequest.stats:type_name -> milvus.proto.common.SegmentStats - 130, // 145: milvus.proto.data.UpdateChannelCheckpointRequest.base:type_name -> milvus.proto.common.MsgBase - 134, // 146: milvus.proto.data.UpdateChannelCheckpointRequest.position:type_name -> milvus.proto.msg.MsgPosition - 134, // 147: milvus.proto.data.UpdateChannelCheckpointRequest.channel_checkpoints:type_name -> milvus.proto.msg.MsgPosition - 130, // 148: milvus.proto.data.ResendSegmentStatsRequest.base:type_name -> milvus.proto.common.MsgBase - 131, // 149: milvus.proto.data.ResendSegmentStatsResponse.status:type_name -> milvus.proto.common.Status - 130, // 150: milvus.proto.data.MarkSegmentsDroppedRequest.base:type_name -> milvus.proto.common.MsgBase - 137, // 151: milvus.proto.data.AlterCollectionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 138, // 152: milvus.proto.data.AlterCollectionRequest.start_positions:type_name -> milvus.proto.common.KeyDataPair - 136, // 153: milvus.proto.data.AlterCollectionRequest.properties:type_name -> milvus.proto.common.KeyValuePair - 140, // 154: milvus.proto.data.AddCollectionFieldRequest.field_schema:type_name -> milvus.proto.schema.FieldSchema - 137, // 155: milvus.proto.data.AddCollectionFieldRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 138, // 156: milvus.proto.data.AddCollectionFieldRequest.start_positions:type_name -> milvus.proto.common.KeyDataPair - 136, // 157: milvus.proto.data.AddCollectionFieldRequest.properties:type_name -> milvus.proto.common.KeyValuePair - 131, // 158: milvus.proto.data.GcConfirmResponse.status:type_name -> milvus.proto.common.Status - 130, // 159: milvus.proto.data.ReportDataNodeTtMsgsRequest.base:type_name -> milvus.proto.common.MsgBase - 141, // 160: milvus.proto.data.ReportDataNodeTtMsgsRequest.msgs:type_name -> milvus.proto.msg.DataNodeTtMsg + 138, // 140: milvus.proto.data.DropVirtualChannelSegment.startPosition:type_name -> milvus.proto.msg.MsgPosition + 138, // 141: milvus.proto.data.DropVirtualChannelSegment.checkPoint:type_name -> milvus.proto.msg.MsgPosition + 135, // 142: milvus.proto.data.DropVirtualChannelResponse.status:type_name -> milvus.proto.common.Status + 134, // 143: milvus.proto.data.UpdateSegmentStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase + 143, // 144: milvus.proto.data.UpdateSegmentStatisticsRequest.stats:type_name -> milvus.proto.common.SegmentStats + 134, // 145: milvus.proto.data.UpdateChannelCheckpointRequest.base:type_name -> milvus.proto.common.MsgBase + 138, // 146: milvus.proto.data.UpdateChannelCheckpointRequest.position:type_name -> milvus.proto.msg.MsgPosition + 138, // 147: milvus.proto.data.UpdateChannelCheckpointRequest.channel_checkpoints:type_name -> milvus.proto.msg.MsgPosition + 134, // 148: milvus.proto.data.ResendSegmentStatsRequest.base:type_name -> milvus.proto.common.MsgBase + 135, // 149: milvus.proto.data.ResendSegmentStatsResponse.status:type_name -> milvus.proto.common.Status + 134, // 150: milvus.proto.data.MarkSegmentsDroppedRequest.base:type_name -> milvus.proto.common.MsgBase + 141, // 151: milvus.proto.data.AlterCollectionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 142, // 152: milvus.proto.data.AlterCollectionRequest.start_positions:type_name -> milvus.proto.common.KeyDataPair + 140, // 153: milvus.proto.data.AlterCollectionRequest.properties:type_name -> milvus.proto.common.KeyValuePair + 144, // 154: milvus.proto.data.AddCollectionFieldRequest.field_schema:type_name -> milvus.proto.schema.FieldSchema + 141, // 155: milvus.proto.data.AddCollectionFieldRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 142, // 156: milvus.proto.data.AddCollectionFieldRequest.start_positions:type_name -> milvus.proto.common.KeyDataPair + 140, // 157: milvus.proto.data.AddCollectionFieldRequest.properties:type_name -> milvus.proto.common.KeyValuePair + 135, // 158: milvus.proto.data.GcConfirmResponse.status:type_name -> milvus.proto.common.Status + 134, // 159: milvus.proto.data.ReportDataNodeTtMsgsRequest.base:type_name -> milvus.proto.common.MsgBase + 145, // 160: milvus.proto.data.ReportDataNodeTtMsgsRequest.msgs:type_name -> milvus.proto.msg.DataNodeTtMsg 62, // 161: milvus.proto.data.ChannelOperationsRequest.infos:type_name -> milvus.proto.data.ChannelWatchInfo - 131, // 162: milvus.proto.data.ChannelOperationProgressResponse.status:type_name -> milvus.proto.common.Status + 135, // 162: milvus.proto.data.ChannelOperationProgressResponse.status:type_name -> milvus.proto.common.Status 2, // 163: milvus.proto.data.ChannelOperationProgressResponse.state:type_name -> milvus.proto.data.ChannelWatchState - 137, // 164: milvus.proto.data.PreImportRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 142, // 165: milvus.proto.data.PreImportRequest.import_files:type_name -> milvus.proto.internal.ImportFile - 136, // 166: milvus.proto.data.PreImportRequest.options:type_name -> milvus.proto.common.KeyValuePair - 143, // 167: milvus.proto.data.PreImportRequest.storage_config:type_name -> milvus.proto.index.StorageConfig - 136, // 168: milvus.proto.data.PreImportRequest.plugin_context:type_name -> milvus.proto.common.KeyValuePair - 137, // 169: milvus.proto.data.ImportRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 142, // 170: milvus.proto.data.ImportRequest.files:type_name -> milvus.proto.internal.ImportFile - 136, // 171: milvus.proto.data.ImportRequest.options:type_name -> milvus.proto.common.KeyValuePair + 141, // 164: milvus.proto.data.PreImportRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 146, // 165: milvus.proto.data.PreImportRequest.import_files:type_name -> milvus.proto.internal.ImportFile + 140, // 166: milvus.proto.data.PreImportRequest.options:type_name -> milvus.proto.common.KeyValuePair + 147, // 167: milvus.proto.data.PreImportRequest.storage_config:type_name -> milvus.proto.index.StorageConfig + 140, // 168: milvus.proto.data.PreImportRequest.plugin_context:type_name -> milvus.proto.common.KeyValuePair + 141, // 169: milvus.proto.data.ImportRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 146, // 170: milvus.proto.data.ImportRequest.files:type_name -> milvus.proto.internal.ImportFile + 140, // 171: milvus.proto.data.ImportRequest.options:type_name -> milvus.proto.common.KeyValuePair 94, // 172: milvus.proto.data.ImportRequest.ID_range:type_name -> milvus.proto.data.IDRange 95, // 173: milvus.proto.data.ImportRequest.request_segments:type_name -> milvus.proto.data.ImportRequestSegment - 143, // 174: milvus.proto.data.ImportRequest.storage_config:type_name -> milvus.proto.index.StorageConfig - 136, // 175: milvus.proto.data.ImportRequest.plugin_context:type_name -> milvus.proto.common.KeyValuePair - 127, // 176: milvus.proto.data.PartitionImportStats.partition_rows:type_name -> milvus.proto.data.PartitionImportStats.PartitionRowsEntry - 128, // 177: milvus.proto.data.PartitionImportStats.partition_data_size:type_name -> milvus.proto.data.PartitionImportStats.PartitionDataSizeEntry - 142, // 178: milvus.proto.data.ImportFileStats.import_file:type_name -> milvus.proto.internal.ImportFile - 129, // 179: milvus.proto.data.ImportFileStats.hashed_stats:type_name -> milvus.proto.data.ImportFileStats.HashedStatsEntry - 131, // 180: milvus.proto.data.QueryPreImportResponse.status:type_name -> milvus.proto.common.Status + 147, // 174: milvus.proto.data.ImportRequest.storage_config:type_name -> milvus.proto.index.StorageConfig + 140, // 175: milvus.proto.data.ImportRequest.plugin_context:type_name -> milvus.proto.common.KeyValuePair + 131, // 176: milvus.proto.data.PartitionImportStats.partition_rows:type_name -> milvus.proto.data.PartitionImportStats.PartitionRowsEntry + 132, // 177: milvus.proto.data.PartitionImportStats.partition_data_size:type_name -> milvus.proto.data.PartitionImportStats.PartitionDataSizeEntry + 146, // 178: milvus.proto.data.ImportFileStats.import_file:type_name -> milvus.proto.internal.ImportFile + 133, // 179: milvus.proto.data.ImportFileStats.hashed_stats:type_name -> milvus.proto.data.ImportFileStats.HashedStatsEntry + 135, // 180: milvus.proto.data.QueryPreImportResponse.status:type_name -> milvus.proto.common.Status 4, // 181: milvus.proto.data.QueryPreImportResponse.state:type_name -> milvus.proto.data.ImportTaskStateV2 99, // 182: milvus.proto.data.QueryPreImportResponse.file_stats:type_name -> milvus.proto.data.ImportFileStats 46, // 183: milvus.proto.data.ImportSegmentInfo.binlogs:type_name -> milvus.proto.data.FieldBinlog 46, // 184: milvus.proto.data.ImportSegmentInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog 46, // 185: milvus.proto.data.ImportSegmentInfo.deltalogs:type_name -> milvus.proto.data.FieldBinlog 46, // 186: milvus.proto.data.ImportSegmentInfo.bm25logs:type_name -> milvus.proto.data.FieldBinlog - 131, // 187: milvus.proto.data.QueryImportResponse.status:type_name -> milvus.proto.common.Status + 135, // 187: milvus.proto.data.QueryImportResponse.status:type_name -> milvus.proto.common.Status 4, // 188: milvus.proto.data.QueryImportResponse.state:type_name -> milvus.proto.data.ImportTaskStateV2 102, // 189: milvus.proto.data.QueryImportResponse.import_segments_info:type_name -> milvus.proto.data.ImportSegmentInfo - 137, // 190: milvus.proto.data.ImportJob.schema:type_name -> milvus.proto.schema.CollectionSchema - 144, // 191: milvus.proto.data.ImportJob.state:type_name -> milvus.proto.internal.ImportJobState - 142, // 192: milvus.proto.data.ImportJob.files:type_name -> milvus.proto.internal.ImportFile - 136, // 193: milvus.proto.data.ImportJob.options:type_name -> milvus.proto.common.KeyValuePair + 141, // 190: milvus.proto.data.ImportJob.schema:type_name -> milvus.proto.schema.CollectionSchema + 148, // 191: milvus.proto.data.ImportJob.state:type_name -> milvus.proto.internal.ImportJobState + 146, // 192: milvus.proto.data.ImportJob.files:type_name -> milvus.proto.internal.ImportFile + 140, // 193: milvus.proto.data.ImportJob.options:type_name -> milvus.proto.common.KeyValuePair 4, // 194: milvus.proto.data.PreImportTask.state:type_name -> milvus.proto.data.ImportTaskStateV2 99, // 195: milvus.proto.data.PreImportTask.file_stats:type_name -> milvus.proto.data.ImportFileStats 4, // 196: milvus.proto.data.ImportTaskV2.state:type_name -> milvus.proto.data.ImportTaskStateV2 99, // 197: milvus.proto.data.ImportTaskV2.file_stats:type_name -> milvus.proto.data.ImportFileStats 5, // 198: milvus.proto.data.ImportTaskV2.source:type_name -> milvus.proto.data.ImportTaskSourceV2 - 130, // 199: milvus.proto.data.GcControlRequest.base:type_name -> milvus.proto.common.MsgBase + 134, // 199: milvus.proto.data.GcControlRequest.base:type_name -> milvus.proto.common.MsgBase 6, // 200: milvus.proto.data.GcControlRequest.command:type_name -> milvus.proto.data.GcCommand - 136, // 201: milvus.proto.data.GcControlRequest.params:type_name -> milvus.proto.common.KeyValuePair - 131, // 202: milvus.proto.data.QuerySlotResponse.status:type_name -> milvus.proto.common.Status + 140, // 201: milvus.proto.data.GcControlRequest.params:type_name -> milvus.proto.common.KeyValuePair + 135, // 202: milvus.proto.data.QuerySlotResponse.status:type_name -> milvus.proto.common.Status 3, // 203: milvus.proto.data.CompactionTask.type:type_name -> milvus.proto.data.CompactionType 7, // 204: milvus.proto.data.CompactionTask.state:type_name -> milvus.proto.data.CompactionTaskState - 134, // 205: milvus.proto.data.CompactionTask.pos:type_name -> milvus.proto.msg.MsgPosition - 137, // 206: milvus.proto.data.CompactionTask.schema:type_name -> milvus.proto.schema.CollectionSchema - 140, // 207: milvus.proto.data.CompactionTask.clustering_key_field:type_name -> milvus.proto.schema.FieldSchema + 138, // 205: milvus.proto.data.CompactionTask.pos:type_name -> milvus.proto.msg.MsgPosition + 141, // 206: milvus.proto.data.CompactionTask.schema:type_name -> milvus.proto.schema.CollectionSchema + 144, // 207: milvus.proto.data.CompactionTask.clustering_key_field:type_name -> milvus.proto.schema.FieldSchema 94, // 208: milvus.proto.data.CompactionTask.pre_allocated_segmentIDs:type_name -> milvus.proto.data.IDRange - 131, // 209: milvus.proto.data.CreateExternalCollectionResponse.status:type_name -> milvus.proto.common.Status - 134, // 210: milvus.proto.data.FlushResponse.ChannelCpsEntry.value:type_name -> milvus.proto.msg.MsgPosition - 134, // 211: milvus.proto.data.FlushResult.ChannelCpsEntry.value:type_name -> milvus.proto.msg.MsgPosition - 145, // 212: milvus.proto.data.FlushAllResponse.FlushAllMsgsEntry.value:type_name -> milvus.proto.common.ImmutableMessage - 134, // 213: milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry.value:type_name -> milvus.proto.msg.MsgPosition - 47, // 214: milvus.proto.data.SegmentInfo.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats - 48, // 215: milvus.proto.data.SegmentInfo.JsonKeyStatsEntry.value:type_name -> milvus.proto.data.JsonKeyStats - 47, // 216: milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats - 64, // 217: milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry.value:type_name -> milvus.proto.data.SyncSegmentInfo - 47, // 218: milvus.proto.data.CompactionSegment.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats - 98, // 219: milvus.proto.data.ImportFileStats.HashedStatsEntry.value:type_name -> milvus.proto.data.PartitionImportStats - 9, // 220: milvus.proto.data.DataCoord.Flush:input_type -> milvus.proto.data.FlushRequest - 12, // 221: milvus.proto.data.DataCoord.FlushAll:input_type -> milvus.proto.data.FlushAllRequest - 146, // 222: milvus.proto.data.DataCoord.CreateExternalCollection:input_type -> milvus.proto.msg.CreateCollectionRequest - 17, // 223: milvus.proto.data.DataCoord.AllocSegment:input_type -> milvus.proto.data.AllocSegmentRequest - 19, // 224: milvus.proto.data.DataCoord.AssignSegmentID:input_type -> milvus.proto.data.AssignSegmentIDRequest - 25, // 225: milvus.proto.data.DataCoord.GetSegmentInfo:input_type -> milvus.proto.data.GetSegmentInfoRequest - 22, // 226: milvus.proto.data.DataCoord.GetSegmentStates:input_type -> milvus.proto.data.GetSegmentStatesRequest - 27, // 227: milvus.proto.data.DataCoord.GetInsertBinlogPaths:input_type -> milvus.proto.data.GetInsertBinlogPathsRequest - 29, // 228: milvus.proto.data.DataCoord.GetCollectionStatistics:input_type -> milvus.proto.data.GetCollectionStatisticsRequest - 31, // 229: milvus.proto.data.DataCoord.GetPartitionStatistics:input_type -> milvus.proto.data.GetPartitionStatisticsRequest - 33, // 230: milvus.proto.data.DataCoord.GetSegmentInfoChannel:input_type -> milvus.proto.data.GetSegmentInfoChannelRequest - 40, // 231: milvus.proto.data.DataCoord.SaveBinlogPaths:input_type -> milvus.proto.data.SaveBinlogPathsRequest - 51, // 232: milvus.proto.data.DataCoord.GetRecoveryInfo:input_type -> milvus.proto.data.GetRecoveryInfoRequest - 53, // 233: milvus.proto.data.DataCoord.GetRecoveryInfoV2:input_type -> milvus.proto.data.GetRecoveryInfoRequestV2 - 54, // 234: milvus.proto.data.DataCoord.GetChannelRecoveryInfo:input_type -> milvus.proto.data.GetChannelRecoveryInfoRequest - 59, // 235: milvus.proto.data.DataCoord.GetFlushedSegments:input_type -> milvus.proto.data.GetFlushedSegmentsRequest - 57, // 236: milvus.proto.data.DataCoord.GetSegmentsByStates:input_type -> milvus.proto.data.GetSegmentsByStatesRequest - 147, // 237: milvus.proto.data.DataCoord.GetFlushAllState:input_type -> milvus.proto.milvus.GetFlushAllStateRequest - 148, // 238: milvus.proto.data.DataCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest - 149, // 239: milvus.proto.data.DataCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest - 150, // 240: milvus.proto.data.DataCoord.ManualCompaction:input_type -> milvus.proto.milvus.ManualCompactionRequest - 151, // 241: milvus.proto.data.DataCoord.GetCompactionState:input_type -> milvus.proto.milvus.GetCompactionStateRequest - 152, // 242: milvus.proto.data.DataCoord.GetCompactionStateWithPlans:input_type -> milvus.proto.milvus.GetCompactionPlansRequest - 72, // 243: milvus.proto.data.DataCoord.WatchChannels:input_type -> milvus.proto.data.WatchChannelsRequest - 90, // 244: milvus.proto.data.DataCoord.GetFlushState:input_type -> milvus.proto.data.GetFlushStateRequest - 76, // 245: milvus.proto.data.DataCoord.DropVirtualChannel:input_type -> milvus.proto.data.DropVirtualChannelRequest - 74, // 246: milvus.proto.data.DataCoord.SetSegmentState:input_type -> milvus.proto.data.SetSegmentStateRequest - 79, // 247: milvus.proto.data.DataCoord.UpdateSegmentStatistics:input_type -> milvus.proto.data.UpdateSegmentStatisticsRequest - 80, // 248: milvus.proto.data.DataCoord.UpdateChannelCheckpoint:input_type -> milvus.proto.data.UpdateChannelCheckpointRequest - 83, // 249: milvus.proto.data.DataCoord.MarkSegmentsDropped:input_type -> milvus.proto.data.MarkSegmentsDroppedRequest - 85, // 250: milvus.proto.data.DataCoord.BroadcastAlteredCollection:input_type -> milvus.proto.data.AlterCollectionRequest - 153, // 251: milvus.proto.data.DataCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest - 154, // 252: milvus.proto.data.DataCoord.CreateIndex:input_type -> milvus.proto.index.CreateIndexRequest - 155, // 253: milvus.proto.data.DataCoord.AlterIndex:input_type -> milvus.proto.index.AlterIndexRequest - 156, // 254: milvus.proto.data.DataCoord.GetIndexState:input_type -> milvus.proto.index.GetIndexStateRequest - 157, // 255: milvus.proto.data.DataCoord.GetSegmentIndexState:input_type -> milvus.proto.index.GetSegmentIndexStateRequest - 158, // 256: milvus.proto.data.DataCoord.GetIndexInfos:input_type -> milvus.proto.index.GetIndexInfoRequest - 159, // 257: milvus.proto.data.DataCoord.DropIndex:input_type -> milvus.proto.index.DropIndexRequest - 160, // 258: milvus.proto.data.DataCoord.DescribeIndex:input_type -> milvus.proto.index.DescribeIndexRequest - 161, // 259: milvus.proto.data.DataCoord.GetIndexStatistics:input_type -> milvus.proto.index.GetIndexStatisticsRequest - 162, // 260: milvus.proto.data.DataCoord.GetIndexBuildProgress:input_type -> milvus.proto.index.GetIndexBuildProgressRequest - 163, // 261: milvus.proto.data.DataCoord.ListIndexes:input_type -> milvus.proto.index.ListIndexesRequest - 87, // 262: milvus.proto.data.DataCoord.GcConfirm:input_type -> milvus.proto.data.GcConfirmRequest - 89, // 263: milvus.proto.data.DataCoord.ReportDataNodeTtMsgs:input_type -> milvus.proto.data.ReportDataNodeTtMsgsRequest - 108, // 264: milvus.proto.data.DataCoord.GcControl:input_type -> milvus.proto.data.GcControlRequest - 164, // 265: milvus.proto.data.DataCoord.ImportV2:input_type -> milvus.proto.internal.ImportRequestInternal - 165, // 266: milvus.proto.data.DataCoord.GetImportProgress:input_type -> milvus.proto.internal.GetImportProgressRequest - 166, // 267: milvus.proto.data.DataCoord.ListImports:input_type -> milvus.proto.internal.ListImportsRequestInternal - 167, // 268: milvus.proto.data.DataCoord.AddFileResource:input_type -> milvus.proto.milvus.AddFileResourceRequest - 168, // 269: milvus.proto.data.DataCoord.RemoveFileResource:input_type -> milvus.proto.milvus.RemoveFileResourceRequest - 169, // 270: milvus.proto.data.DataCoord.ListFileResources:input_type -> milvus.proto.milvus.ListFileResourcesRequest - 170, // 271: milvus.proto.data.DataNode.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest - 171, // 272: milvus.proto.data.DataNode.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest - 35, // 273: milvus.proto.data.DataNode.WatchDmChannels:input_type -> milvus.proto.data.WatchDmChannelsRequest - 36, // 274: milvus.proto.data.DataNode.FlushSegments:input_type -> milvus.proto.data.FlushSegmentsRequest - 148, // 275: milvus.proto.data.DataNode.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest - 149, // 276: milvus.proto.data.DataNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest - 67, // 277: milvus.proto.data.DataNode.CompactionV2:input_type -> milvus.proto.data.CompactionPlan - 63, // 278: milvus.proto.data.DataNode.GetCompactionState:input_type -> milvus.proto.data.CompactionStateRequest - 65, // 279: milvus.proto.data.DataNode.SyncSegments:input_type -> milvus.proto.data.SyncSegmentsRequest - 81, // 280: milvus.proto.data.DataNode.ResendSegmentStats:input_type -> milvus.proto.data.ResendSegmentStatsRequest - 15, // 281: milvus.proto.data.DataNode.FlushChannels:input_type -> milvus.proto.data.FlushChannelsRequest - 91, // 282: milvus.proto.data.DataNode.NotifyChannelOperation:input_type -> milvus.proto.data.ChannelOperationsRequest - 62, // 283: milvus.proto.data.DataNode.CheckChannelOperationProgress:input_type -> milvus.proto.data.ChannelWatchInfo - 93, // 284: milvus.proto.data.DataNode.PreImport:input_type -> milvus.proto.data.PreImportRequest - 96, // 285: milvus.proto.data.DataNode.ImportV2:input_type -> milvus.proto.data.ImportRequest - 97, // 286: milvus.proto.data.DataNode.QueryPreImport:input_type -> milvus.proto.data.QueryPreImportRequest - 101, // 287: milvus.proto.data.DataNode.QueryImport:input_type -> milvus.proto.data.QueryImportRequest - 104, // 288: milvus.proto.data.DataNode.DropImport:input_type -> milvus.proto.data.DropImportRequest - 110, // 289: milvus.proto.data.DataNode.QuerySlot:input_type -> milvus.proto.data.QuerySlotRequest - 114, // 290: milvus.proto.data.DataNode.DropCompactionPlan:input_type -> milvus.proto.data.DropCompactionPlanRequest - 172, // 291: milvus.proto.data.DataNode.SyncFileResource:input_type -> milvus.proto.internal.SyncFileResourceRequest - 10, // 292: milvus.proto.data.DataCoord.Flush:output_type -> milvus.proto.data.FlushResponse - 14, // 293: milvus.proto.data.DataCoord.FlushAll:output_type -> milvus.proto.data.FlushAllResponse - 116, // 294: milvus.proto.data.DataCoord.CreateExternalCollection:output_type -> milvus.proto.data.CreateExternalCollectionResponse - 18, // 295: milvus.proto.data.DataCoord.AllocSegment:output_type -> milvus.proto.data.AllocSegmentResponse - 21, // 296: milvus.proto.data.DataCoord.AssignSegmentID:output_type -> milvus.proto.data.AssignSegmentIDResponse - 26, // 297: milvus.proto.data.DataCoord.GetSegmentInfo:output_type -> milvus.proto.data.GetSegmentInfoResponse - 24, // 298: milvus.proto.data.DataCoord.GetSegmentStates:output_type -> milvus.proto.data.GetSegmentStatesResponse - 28, // 299: milvus.proto.data.DataCoord.GetInsertBinlogPaths:output_type -> milvus.proto.data.GetInsertBinlogPathsResponse - 30, // 300: milvus.proto.data.DataCoord.GetCollectionStatistics:output_type -> milvus.proto.data.GetCollectionStatisticsResponse - 32, // 301: milvus.proto.data.DataCoord.GetPartitionStatistics:output_type -> milvus.proto.data.GetPartitionStatisticsResponse - 173, // 302: milvus.proto.data.DataCoord.GetSegmentInfoChannel:output_type -> milvus.proto.milvus.StringResponse - 131, // 303: milvus.proto.data.DataCoord.SaveBinlogPaths:output_type -> milvus.proto.common.Status - 50, // 304: milvus.proto.data.DataCoord.GetRecoveryInfo:output_type -> milvus.proto.data.GetRecoveryInfoResponse - 52, // 305: milvus.proto.data.DataCoord.GetRecoveryInfoV2:output_type -> milvus.proto.data.GetRecoveryInfoResponseV2 - 55, // 306: milvus.proto.data.DataCoord.GetChannelRecoveryInfo:output_type -> milvus.proto.data.GetChannelRecoveryInfoResponse - 60, // 307: milvus.proto.data.DataCoord.GetFlushedSegments:output_type -> milvus.proto.data.GetFlushedSegmentsResponse - 58, // 308: milvus.proto.data.DataCoord.GetSegmentsByStates:output_type -> milvus.proto.data.GetSegmentsByStatesResponse - 174, // 309: milvus.proto.data.DataCoord.GetFlushAllState:output_type -> milvus.proto.milvus.GetFlushAllStateResponse - 175, // 310: milvus.proto.data.DataCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 176, // 311: milvus.proto.data.DataCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 177, // 312: milvus.proto.data.DataCoord.ManualCompaction:output_type -> milvus.proto.milvus.ManualCompactionResponse - 178, // 313: milvus.proto.data.DataCoord.GetCompactionState:output_type -> milvus.proto.milvus.GetCompactionStateResponse - 179, // 314: milvus.proto.data.DataCoord.GetCompactionStateWithPlans:output_type -> milvus.proto.milvus.GetCompactionPlansResponse - 73, // 315: milvus.proto.data.DataCoord.WatchChannels:output_type -> milvus.proto.data.WatchChannelsResponse - 180, // 316: milvus.proto.data.DataCoord.GetFlushState:output_type -> milvus.proto.milvus.GetFlushStateResponse - 78, // 317: milvus.proto.data.DataCoord.DropVirtualChannel:output_type -> milvus.proto.data.DropVirtualChannelResponse - 75, // 318: milvus.proto.data.DataCoord.SetSegmentState:output_type -> milvus.proto.data.SetSegmentStateResponse - 131, // 319: milvus.proto.data.DataCoord.UpdateSegmentStatistics:output_type -> milvus.proto.common.Status - 131, // 320: milvus.proto.data.DataCoord.UpdateChannelCheckpoint:output_type -> milvus.proto.common.Status - 131, // 321: milvus.proto.data.DataCoord.MarkSegmentsDropped:output_type -> milvus.proto.common.Status - 131, // 322: milvus.proto.data.DataCoord.BroadcastAlteredCollection:output_type -> milvus.proto.common.Status - 181, // 323: milvus.proto.data.DataCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse - 131, // 324: milvus.proto.data.DataCoord.CreateIndex:output_type -> milvus.proto.common.Status - 131, // 325: milvus.proto.data.DataCoord.AlterIndex:output_type -> milvus.proto.common.Status - 182, // 326: milvus.proto.data.DataCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse - 183, // 327: milvus.proto.data.DataCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse - 184, // 328: milvus.proto.data.DataCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse - 131, // 329: milvus.proto.data.DataCoord.DropIndex:output_type -> milvus.proto.common.Status - 185, // 330: milvus.proto.data.DataCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse - 186, // 331: milvus.proto.data.DataCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse - 187, // 332: milvus.proto.data.DataCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse - 188, // 333: milvus.proto.data.DataCoord.ListIndexes:output_type -> milvus.proto.index.ListIndexesResponse - 88, // 334: milvus.proto.data.DataCoord.GcConfirm:output_type -> milvus.proto.data.GcConfirmResponse - 131, // 335: milvus.proto.data.DataCoord.ReportDataNodeTtMsgs:output_type -> milvus.proto.common.Status - 131, // 336: milvus.proto.data.DataCoord.GcControl:output_type -> milvus.proto.common.Status - 189, // 337: milvus.proto.data.DataCoord.ImportV2:output_type -> milvus.proto.internal.ImportResponse - 190, // 338: milvus.proto.data.DataCoord.GetImportProgress:output_type -> milvus.proto.internal.GetImportProgressResponse - 191, // 339: milvus.proto.data.DataCoord.ListImports:output_type -> milvus.proto.internal.ListImportsResponse - 131, // 340: milvus.proto.data.DataCoord.AddFileResource:output_type -> milvus.proto.common.Status - 131, // 341: milvus.proto.data.DataCoord.RemoveFileResource:output_type -> milvus.proto.common.Status - 192, // 342: milvus.proto.data.DataCoord.ListFileResources:output_type -> milvus.proto.milvus.ListFileResourcesResponse - 193, // 343: milvus.proto.data.DataNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates - 173, // 344: milvus.proto.data.DataNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse - 131, // 345: milvus.proto.data.DataNode.WatchDmChannels:output_type -> milvus.proto.common.Status - 131, // 346: milvus.proto.data.DataNode.FlushSegments:output_type -> milvus.proto.common.Status - 175, // 347: milvus.proto.data.DataNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 176, // 348: milvus.proto.data.DataNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 131, // 349: milvus.proto.data.DataNode.CompactionV2:output_type -> milvus.proto.common.Status - 70, // 350: milvus.proto.data.DataNode.GetCompactionState:output_type -> milvus.proto.data.CompactionStateResponse - 131, // 351: milvus.proto.data.DataNode.SyncSegments:output_type -> milvus.proto.common.Status - 82, // 352: milvus.proto.data.DataNode.ResendSegmentStats:output_type -> milvus.proto.data.ResendSegmentStatsResponse - 131, // 353: milvus.proto.data.DataNode.FlushChannels:output_type -> milvus.proto.common.Status - 131, // 354: milvus.proto.data.DataNode.NotifyChannelOperation:output_type -> milvus.proto.common.Status - 92, // 355: milvus.proto.data.DataNode.CheckChannelOperationProgress:output_type -> milvus.proto.data.ChannelOperationProgressResponse - 131, // 356: milvus.proto.data.DataNode.PreImport:output_type -> milvus.proto.common.Status - 131, // 357: milvus.proto.data.DataNode.ImportV2:output_type -> milvus.proto.common.Status - 100, // 358: milvus.proto.data.DataNode.QueryPreImport:output_type -> milvus.proto.data.QueryPreImportResponse - 103, // 359: milvus.proto.data.DataNode.QueryImport:output_type -> milvus.proto.data.QueryImportResponse - 131, // 360: milvus.proto.data.DataNode.DropImport:output_type -> milvus.proto.common.Status - 111, // 361: milvus.proto.data.DataNode.QuerySlot:output_type -> milvus.proto.data.QuerySlotResponse - 131, // 362: milvus.proto.data.DataNode.DropCompactionPlan:output_type -> milvus.proto.common.Status - 131, // 363: milvus.proto.data.DataNode.SyncFileResource:output_type -> milvus.proto.common.Status - 292, // [292:364] is the sub-list for method output_type - 220, // [220:292] is the sub-list for method input_type - 220, // [220:220] is the sub-list for extension type_name - 220, // [220:220] is the sub-list for extension extendee - 0, // [0:220] is the sub-list for field type_name + 135, // 209: milvus.proto.data.CreateExternalCollectionResponse.status:type_name -> milvus.proto.common.Status + 134, // 210: milvus.proto.data.UpdateExternalCollectionRequest.base:type_name -> milvus.proto.common.MsgBase + 38, // 211: milvus.proto.data.UpdateExternalCollectionRequest.currentSegments:type_name -> milvus.proto.data.SegmentInfo + 135, // 212: milvus.proto.data.UpdateExternalCollectionResponse.status:type_name -> milvus.proto.common.Status + 38, // 213: milvus.proto.data.UpdateExternalCollectionResponse.updatedSegments:type_name -> milvus.proto.data.SegmentInfo + 149, // 214: milvus.proto.data.UpdateExternalCollectionResponse.state:type_name -> milvus.proto.index.JobState + 138, // 215: milvus.proto.data.FlushResponse.ChannelCpsEntry.value:type_name -> milvus.proto.msg.MsgPosition + 138, // 216: milvus.proto.data.FlushResult.ChannelCpsEntry.value:type_name -> milvus.proto.msg.MsgPosition + 150, // 217: milvus.proto.data.FlushAllResponse.FlushAllMsgsEntry.value:type_name -> milvus.proto.common.ImmutableMessage + 138, // 218: milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry.value:type_name -> milvus.proto.msg.MsgPosition + 47, // 219: milvus.proto.data.SegmentInfo.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats + 48, // 220: milvus.proto.data.SegmentInfo.JsonKeyStatsEntry.value:type_name -> milvus.proto.data.JsonKeyStats + 47, // 221: milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats + 64, // 222: milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry.value:type_name -> milvus.proto.data.SyncSegmentInfo + 47, // 223: milvus.proto.data.CompactionSegment.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats + 98, // 224: milvus.proto.data.ImportFileStats.HashedStatsEntry.value:type_name -> milvus.proto.data.PartitionImportStats + 9, // 225: milvus.proto.data.DataCoord.Flush:input_type -> milvus.proto.data.FlushRequest + 12, // 226: milvus.proto.data.DataCoord.FlushAll:input_type -> milvus.proto.data.FlushAllRequest + 151, // 227: milvus.proto.data.DataCoord.CreateExternalCollection:input_type -> milvus.proto.msg.CreateCollectionRequest + 17, // 228: milvus.proto.data.DataCoord.AllocSegment:input_type -> milvus.proto.data.AllocSegmentRequest + 19, // 229: milvus.proto.data.DataCoord.AssignSegmentID:input_type -> milvus.proto.data.AssignSegmentIDRequest + 25, // 230: milvus.proto.data.DataCoord.GetSegmentInfo:input_type -> milvus.proto.data.GetSegmentInfoRequest + 22, // 231: milvus.proto.data.DataCoord.GetSegmentStates:input_type -> milvus.proto.data.GetSegmentStatesRequest + 27, // 232: milvus.proto.data.DataCoord.GetInsertBinlogPaths:input_type -> milvus.proto.data.GetInsertBinlogPathsRequest + 29, // 233: milvus.proto.data.DataCoord.GetCollectionStatistics:input_type -> milvus.proto.data.GetCollectionStatisticsRequest + 31, // 234: milvus.proto.data.DataCoord.GetPartitionStatistics:input_type -> milvus.proto.data.GetPartitionStatisticsRequest + 33, // 235: milvus.proto.data.DataCoord.GetSegmentInfoChannel:input_type -> milvus.proto.data.GetSegmentInfoChannelRequest + 40, // 236: milvus.proto.data.DataCoord.SaveBinlogPaths:input_type -> milvus.proto.data.SaveBinlogPathsRequest + 51, // 237: milvus.proto.data.DataCoord.GetRecoveryInfo:input_type -> milvus.proto.data.GetRecoveryInfoRequest + 53, // 238: milvus.proto.data.DataCoord.GetRecoveryInfoV2:input_type -> milvus.proto.data.GetRecoveryInfoRequestV2 + 54, // 239: milvus.proto.data.DataCoord.GetChannelRecoveryInfo:input_type -> milvus.proto.data.GetChannelRecoveryInfoRequest + 59, // 240: milvus.proto.data.DataCoord.GetFlushedSegments:input_type -> milvus.proto.data.GetFlushedSegmentsRequest + 57, // 241: milvus.proto.data.DataCoord.GetSegmentsByStates:input_type -> milvus.proto.data.GetSegmentsByStatesRequest + 152, // 242: milvus.proto.data.DataCoord.GetFlushAllState:input_type -> milvus.proto.milvus.GetFlushAllStateRequest + 153, // 243: milvus.proto.data.DataCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 154, // 244: milvus.proto.data.DataCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 155, // 245: milvus.proto.data.DataCoord.ManualCompaction:input_type -> milvus.proto.milvus.ManualCompactionRequest + 156, // 246: milvus.proto.data.DataCoord.GetCompactionState:input_type -> milvus.proto.milvus.GetCompactionStateRequest + 157, // 247: milvus.proto.data.DataCoord.GetCompactionStateWithPlans:input_type -> milvus.proto.milvus.GetCompactionPlansRequest + 72, // 248: milvus.proto.data.DataCoord.WatchChannels:input_type -> milvus.proto.data.WatchChannelsRequest + 90, // 249: milvus.proto.data.DataCoord.GetFlushState:input_type -> milvus.proto.data.GetFlushStateRequest + 76, // 250: milvus.proto.data.DataCoord.DropVirtualChannel:input_type -> milvus.proto.data.DropVirtualChannelRequest + 74, // 251: milvus.proto.data.DataCoord.SetSegmentState:input_type -> milvus.proto.data.SetSegmentStateRequest + 79, // 252: milvus.proto.data.DataCoord.UpdateSegmentStatistics:input_type -> milvus.proto.data.UpdateSegmentStatisticsRequest + 80, // 253: milvus.proto.data.DataCoord.UpdateChannelCheckpoint:input_type -> milvus.proto.data.UpdateChannelCheckpointRequest + 83, // 254: milvus.proto.data.DataCoord.MarkSegmentsDropped:input_type -> milvus.proto.data.MarkSegmentsDroppedRequest + 85, // 255: milvus.proto.data.DataCoord.BroadcastAlteredCollection:input_type -> milvus.proto.data.AlterCollectionRequest + 158, // 256: milvus.proto.data.DataCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest + 159, // 257: milvus.proto.data.DataCoord.CreateIndex:input_type -> milvus.proto.index.CreateIndexRequest + 160, // 258: milvus.proto.data.DataCoord.AlterIndex:input_type -> milvus.proto.index.AlterIndexRequest + 161, // 259: milvus.proto.data.DataCoord.GetIndexState:input_type -> milvus.proto.index.GetIndexStateRequest + 162, // 260: milvus.proto.data.DataCoord.GetSegmentIndexState:input_type -> milvus.proto.index.GetSegmentIndexStateRequest + 163, // 261: milvus.proto.data.DataCoord.GetIndexInfos:input_type -> milvus.proto.index.GetIndexInfoRequest + 164, // 262: milvus.proto.data.DataCoord.DropIndex:input_type -> milvus.proto.index.DropIndexRequest + 165, // 263: milvus.proto.data.DataCoord.DescribeIndex:input_type -> milvus.proto.index.DescribeIndexRequest + 166, // 264: milvus.proto.data.DataCoord.GetIndexStatistics:input_type -> milvus.proto.index.GetIndexStatisticsRequest + 167, // 265: milvus.proto.data.DataCoord.GetIndexBuildProgress:input_type -> milvus.proto.index.GetIndexBuildProgressRequest + 168, // 266: milvus.proto.data.DataCoord.ListIndexes:input_type -> milvus.proto.index.ListIndexesRequest + 87, // 267: milvus.proto.data.DataCoord.GcConfirm:input_type -> milvus.proto.data.GcConfirmRequest + 89, // 268: milvus.proto.data.DataCoord.ReportDataNodeTtMsgs:input_type -> milvus.proto.data.ReportDataNodeTtMsgsRequest + 108, // 269: milvus.proto.data.DataCoord.GcControl:input_type -> milvus.proto.data.GcControlRequest + 169, // 270: milvus.proto.data.DataCoord.ImportV2:input_type -> milvus.proto.internal.ImportRequestInternal + 170, // 271: milvus.proto.data.DataCoord.GetImportProgress:input_type -> milvus.proto.internal.GetImportProgressRequest + 171, // 272: milvus.proto.data.DataCoord.ListImports:input_type -> milvus.proto.internal.ListImportsRequestInternal + 172, // 273: milvus.proto.data.DataCoord.AddFileResource:input_type -> milvus.proto.milvus.AddFileResourceRequest + 173, // 274: milvus.proto.data.DataCoord.RemoveFileResource:input_type -> milvus.proto.milvus.RemoveFileResourceRequest + 174, // 275: milvus.proto.data.DataCoord.ListFileResources:input_type -> milvus.proto.milvus.ListFileResourcesRequest + 175, // 276: milvus.proto.data.DataNode.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 176, // 277: milvus.proto.data.DataNode.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 35, // 278: milvus.proto.data.DataNode.WatchDmChannels:input_type -> milvus.proto.data.WatchDmChannelsRequest + 36, // 279: milvus.proto.data.DataNode.FlushSegments:input_type -> milvus.proto.data.FlushSegmentsRequest + 153, // 280: milvus.proto.data.DataNode.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 154, // 281: milvus.proto.data.DataNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 67, // 282: milvus.proto.data.DataNode.CompactionV2:input_type -> milvus.proto.data.CompactionPlan + 63, // 283: milvus.proto.data.DataNode.GetCompactionState:input_type -> milvus.proto.data.CompactionStateRequest + 65, // 284: milvus.proto.data.DataNode.SyncSegments:input_type -> milvus.proto.data.SyncSegmentsRequest + 81, // 285: milvus.proto.data.DataNode.ResendSegmentStats:input_type -> milvus.proto.data.ResendSegmentStatsRequest + 15, // 286: milvus.proto.data.DataNode.FlushChannels:input_type -> milvus.proto.data.FlushChannelsRequest + 91, // 287: milvus.proto.data.DataNode.NotifyChannelOperation:input_type -> milvus.proto.data.ChannelOperationsRequest + 62, // 288: milvus.proto.data.DataNode.CheckChannelOperationProgress:input_type -> milvus.proto.data.ChannelWatchInfo + 93, // 289: milvus.proto.data.DataNode.PreImport:input_type -> milvus.proto.data.PreImportRequest + 96, // 290: milvus.proto.data.DataNode.ImportV2:input_type -> milvus.proto.data.ImportRequest + 97, // 291: milvus.proto.data.DataNode.QueryPreImport:input_type -> milvus.proto.data.QueryPreImportRequest + 101, // 292: milvus.proto.data.DataNode.QueryImport:input_type -> milvus.proto.data.QueryImportRequest + 104, // 293: milvus.proto.data.DataNode.DropImport:input_type -> milvus.proto.data.DropImportRequest + 110, // 294: milvus.proto.data.DataNode.QuerySlot:input_type -> milvus.proto.data.QuerySlotRequest + 114, // 295: milvus.proto.data.DataNode.DropCompactionPlan:input_type -> milvus.proto.data.DropCompactionPlanRequest + 177, // 296: milvus.proto.data.DataNode.SyncFileResource:input_type -> milvus.proto.internal.SyncFileResourceRequest + 10, // 297: milvus.proto.data.DataCoord.Flush:output_type -> milvus.proto.data.FlushResponse + 14, // 298: milvus.proto.data.DataCoord.FlushAll:output_type -> milvus.proto.data.FlushAllResponse + 116, // 299: milvus.proto.data.DataCoord.CreateExternalCollection:output_type -> milvus.proto.data.CreateExternalCollectionResponse + 18, // 300: milvus.proto.data.DataCoord.AllocSegment:output_type -> milvus.proto.data.AllocSegmentResponse + 21, // 301: milvus.proto.data.DataCoord.AssignSegmentID:output_type -> milvus.proto.data.AssignSegmentIDResponse + 26, // 302: milvus.proto.data.DataCoord.GetSegmentInfo:output_type -> milvus.proto.data.GetSegmentInfoResponse + 24, // 303: milvus.proto.data.DataCoord.GetSegmentStates:output_type -> milvus.proto.data.GetSegmentStatesResponse + 28, // 304: milvus.proto.data.DataCoord.GetInsertBinlogPaths:output_type -> milvus.proto.data.GetInsertBinlogPathsResponse + 30, // 305: milvus.proto.data.DataCoord.GetCollectionStatistics:output_type -> milvus.proto.data.GetCollectionStatisticsResponse + 32, // 306: milvus.proto.data.DataCoord.GetPartitionStatistics:output_type -> milvus.proto.data.GetPartitionStatisticsResponse + 178, // 307: milvus.proto.data.DataCoord.GetSegmentInfoChannel:output_type -> milvus.proto.milvus.StringResponse + 135, // 308: milvus.proto.data.DataCoord.SaveBinlogPaths:output_type -> milvus.proto.common.Status + 50, // 309: milvus.proto.data.DataCoord.GetRecoveryInfo:output_type -> milvus.proto.data.GetRecoveryInfoResponse + 52, // 310: milvus.proto.data.DataCoord.GetRecoveryInfoV2:output_type -> milvus.proto.data.GetRecoveryInfoResponseV2 + 55, // 311: milvus.proto.data.DataCoord.GetChannelRecoveryInfo:output_type -> milvus.proto.data.GetChannelRecoveryInfoResponse + 60, // 312: milvus.proto.data.DataCoord.GetFlushedSegments:output_type -> milvus.proto.data.GetFlushedSegmentsResponse + 58, // 313: milvus.proto.data.DataCoord.GetSegmentsByStates:output_type -> milvus.proto.data.GetSegmentsByStatesResponse + 179, // 314: milvus.proto.data.DataCoord.GetFlushAllState:output_type -> milvus.proto.milvus.GetFlushAllStateResponse + 180, // 315: milvus.proto.data.DataCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 181, // 316: milvus.proto.data.DataCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 182, // 317: milvus.proto.data.DataCoord.ManualCompaction:output_type -> milvus.proto.milvus.ManualCompactionResponse + 183, // 318: milvus.proto.data.DataCoord.GetCompactionState:output_type -> milvus.proto.milvus.GetCompactionStateResponse + 184, // 319: milvus.proto.data.DataCoord.GetCompactionStateWithPlans:output_type -> milvus.proto.milvus.GetCompactionPlansResponse + 73, // 320: milvus.proto.data.DataCoord.WatchChannels:output_type -> milvus.proto.data.WatchChannelsResponse + 185, // 321: milvus.proto.data.DataCoord.GetFlushState:output_type -> milvus.proto.milvus.GetFlushStateResponse + 78, // 322: milvus.proto.data.DataCoord.DropVirtualChannel:output_type -> milvus.proto.data.DropVirtualChannelResponse + 75, // 323: milvus.proto.data.DataCoord.SetSegmentState:output_type -> milvus.proto.data.SetSegmentStateResponse + 135, // 324: milvus.proto.data.DataCoord.UpdateSegmentStatistics:output_type -> milvus.proto.common.Status + 135, // 325: milvus.proto.data.DataCoord.UpdateChannelCheckpoint:output_type -> milvus.proto.common.Status + 135, // 326: milvus.proto.data.DataCoord.MarkSegmentsDropped:output_type -> milvus.proto.common.Status + 135, // 327: milvus.proto.data.DataCoord.BroadcastAlteredCollection:output_type -> milvus.proto.common.Status + 186, // 328: milvus.proto.data.DataCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse + 135, // 329: milvus.proto.data.DataCoord.CreateIndex:output_type -> milvus.proto.common.Status + 135, // 330: milvus.proto.data.DataCoord.AlterIndex:output_type -> milvus.proto.common.Status + 187, // 331: milvus.proto.data.DataCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse + 188, // 332: milvus.proto.data.DataCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse + 189, // 333: milvus.proto.data.DataCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse + 135, // 334: milvus.proto.data.DataCoord.DropIndex:output_type -> milvus.proto.common.Status + 190, // 335: milvus.proto.data.DataCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse + 191, // 336: milvus.proto.data.DataCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse + 192, // 337: milvus.proto.data.DataCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse + 193, // 338: milvus.proto.data.DataCoord.ListIndexes:output_type -> milvus.proto.index.ListIndexesResponse + 88, // 339: milvus.proto.data.DataCoord.GcConfirm:output_type -> milvus.proto.data.GcConfirmResponse + 135, // 340: milvus.proto.data.DataCoord.ReportDataNodeTtMsgs:output_type -> milvus.proto.common.Status + 135, // 341: milvus.proto.data.DataCoord.GcControl:output_type -> milvus.proto.common.Status + 194, // 342: milvus.proto.data.DataCoord.ImportV2:output_type -> milvus.proto.internal.ImportResponse + 195, // 343: milvus.proto.data.DataCoord.GetImportProgress:output_type -> milvus.proto.internal.GetImportProgressResponse + 196, // 344: milvus.proto.data.DataCoord.ListImports:output_type -> milvus.proto.internal.ListImportsResponse + 135, // 345: milvus.proto.data.DataCoord.AddFileResource:output_type -> milvus.proto.common.Status + 135, // 346: milvus.proto.data.DataCoord.RemoveFileResource:output_type -> milvus.proto.common.Status + 197, // 347: milvus.proto.data.DataCoord.ListFileResources:output_type -> milvus.proto.milvus.ListFileResourcesResponse + 198, // 348: milvus.proto.data.DataNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 178, // 349: milvus.proto.data.DataNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 135, // 350: milvus.proto.data.DataNode.WatchDmChannels:output_type -> milvus.proto.common.Status + 135, // 351: milvus.proto.data.DataNode.FlushSegments:output_type -> milvus.proto.common.Status + 180, // 352: milvus.proto.data.DataNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 181, // 353: milvus.proto.data.DataNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 135, // 354: milvus.proto.data.DataNode.CompactionV2:output_type -> milvus.proto.common.Status + 70, // 355: milvus.proto.data.DataNode.GetCompactionState:output_type -> milvus.proto.data.CompactionStateResponse + 135, // 356: milvus.proto.data.DataNode.SyncSegments:output_type -> milvus.proto.common.Status + 82, // 357: milvus.proto.data.DataNode.ResendSegmentStats:output_type -> milvus.proto.data.ResendSegmentStatsResponse + 135, // 358: milvus.proto.data.DataNode.FlushChannels:output_type -> milvus.proto.common.Status + 135, // 359: milvus.proto.data.DataNode.NotifyChannelOperation:output_type -> milvus.proto.common.Status + 92, // 360: milvus.proto.data.DataNode.CheckChannelOperationProgress:output_type -> milvus.proto.data.ChannelOperationProgressResponse + 135, // 361: milvus.proto.data.DataNode.PreImport:output_type -> milvus.proto.common.Status + 135, // 362: milvus.proto.data.DataNode.ImportV2:output_type -> milvus.proto.common.Status + 100, // 363: milvus.proto.data.DataNode.QueryPreImport:output_type -> milvus.proto.data.QueryPreImportResponse + 103, // 364: milvus.proto.data.DataNode.QueryImport:output_type -> milvus.proto.data.QueryImportResponse + 135, // 365: milvus.proto.data.DataNode.DropImport:output_type -> milvus.proto.common.Status + 111, // 366: milvus.proto.data.DataNode.QuerySlot:output_type -> milvus.proto.data.QuerySlotResponse + 135, // 367: milvus.proto.data.DataNode.DropCompactionPlan:output_type -> milvus.proto.common.Status + 135, // 368: milvus.proto.data.DataNode.SyncFileResource:output_type -> milvus.proto.common.Status + 297, // [297:369] is the sub-list for method output_type + 225, // [225:297] is the sub-list for method input_type + 225, // [225:225] is the sub-list for extension type_name + 225, // [225:225] is the sub-list for extension extendee + 0, // [0:225] is the sub-list for field type_name } func init() { file_data_coord_proto_init() } @@ -13452,6 +13766,54 @@ func file_data_coord_proto_init() { return nil } } + file_data_coord_proto_msgTypes[109].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateExternalCollectionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[110].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateExternalCollectionResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[111].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*QueryExternalCollectionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_data_coord_proto_msgTypes[112].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DropExternalCollectionRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } type x struct{} out := protoimpl.TypeBuilder{ @@ -13459,7 +13821,7 @@ func file_data_coord_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_data_coord_proto_rawDesc, NumEnums: 8, - NumMessages: 122, + NumMessages: 126, NumExtensions: 0, NumServices: 2, }, diff --git a/pkg/taskcommon/type.go b/pkg/taskcommon/type.go index 0d0f91257e..26ec54d7e6 100644 --- a/pkg/taskcommon/type.go +++ b/pkg/taskcommon/type.go @@ -19,13 +19,14 @@ package taskcommon type Type = string const ( - TypeNone Type = "None" - PreImport Type = "PreImport" - Import Type = "Import" - Compaction Type = "Compaction" - Index Type = "Index" - Stats Type = "Stats" - Analyze Type = "Analyze" + TypeNone Type = "None" + PreImport Type = "PreImport" + Import Type = "Import" + Compaction Type = "Compaction" + Index Type = "Index" + Stats Type = "Stats" + Analyze Type = "Analyze" + ExternalCollection Type = "ExternalCollection" ) var TypeList = []Type{ @@ -35,4 +36,5 @@ var TypeList = []Type{ Index, Stats, Analyze, + ExternalCollection, }