From 947c8855f3662a77568ad56f3468a17b6c334cf7 Mon Sep 17 00:00:00 2001 From: aoiasd <45024769+aoiasd@users.noreply.github.com> Date: Tue, 18 Nov 2025 16:09:39 +0800 Subject: [PATCH] feat: support search bm25 with highlight (#44923) relate: https://github.com/milvus-io/milvus/issues/42589 --------- Signed-off-by: aoiasd --- .../distributed/querynode/client/client.go | 11 + .../querynode/client/client_test.go | 3 + internal/distributed/querynode/service.go | 4 + .../distributed/querynode/service_test.go | 12 + internal/mocks/mock_querynode.go | 59 + internal/mocks/mock_querynode_client.go | 74 + internal/proxy/search_pipeline.go | 235 +- internal/proxy/search_pipeline_test.go | 103 + internal/proxy/task.go | 91 + internal/proxy/task_search.go | 131 +- internal/proxy/task_search_test.go | 190 ++ internal/proxy/task_test.go | 235 ++ internal/proxy/util.go | 6 + internal/querycoordv2/mocks/mock_querynode.go | 59 + internal/querynodev2/delegator/delegator.go | 1 + .../querynodev2/delegator/delegator_data.go | 55 + .../querynodev2/delegator/delegator_test.go | 153 ++ .../querynodev2/delegator/mock_delegator.go | 59 + internal/querynodev2/services.go | 33 + internal/querynodev2/services_test.go | 43 + internal/util/mock/grpc_querynode_client.go | 4 + internal/util/wrappers/qn_wrapper.go | 4 + pkg/proto/query_coord.proto | 38 + pkg/proto/querypb/query_coord.pb.go | 2107 ++++++++++------- pkg/proto/querypb/query_coord_grpc.pb.go | 37 + 25 files changed, 2878 insertions(+), 869 deletions(-) diff --git a/internal/distributed/querynode/client/client.go b/internal/distributed/querynode/client/client.go index 394c75d770..a39fa9f3f5 100644 --- a/internal/distributed/querynode/client/client.go +++ b/internal/distributed/querynode/client/client.go @@ -405,3 +405,14 @@ func (c *Client) ValidateAnalyzer(ctx context.Context, req *querypb.ValidateAnal return client.ValidateAnalyzer(ctx, req) }) } + +func (c *Client) GetHighlight(ctx context.Context, req *querypb.GetHighlightRequest, _ ...grpc.CallOption) (*querypb.GetHighlightResponse, error) { + req = typeutil.Clone(req) + commonpbutil.UpdateMsgBase( + req.GetBase(), + commonpbutil.FillMsgBaseFromClient(c.nodeID), + ) + return wrapGrpcCall(ctx, c, func(client querypb.QueryNodeClient) (*querypb.GetHighlightResponse, error) { + return client.GetHighlight(ctx, req) + }) +} diff --git a/internal/distributed/querynode/client/client_test.go b/internal/distributed/querynode/client/client_test.go index 1ae5bbb333..3fbfe80993 100644 --- a/internal/distributed/querynode/client/client_test.go +++ b/internal/distributed/querynode/client/client_test.go @@ -117,6 +117,9 @@ func Test_NewClient(t *testing.T) { r23, err := client.ValidateAnalyzer(ctx, nil) retCheck(retNotNil, r23, err) + r24, err := client.GetHighlight(ctx, nil) + retCheck(retNotNil, r24, err) + // stream rpc client, err := client.QueryStream(ctx, nil) retCheck(retNotNil, client, err) diff --git a/internal/distributed/querynode/service.go b/internal/distributed/querynode/service.go index 65addc39e8..aa36497f78 100644 --- a/internal/distributed/querynode/service.go +++ b/internal/distributed/querynode/service.go @@ -411,3 +411,7 @@ func (s *Server) DropIndex(ctx context.Context, req *querypb.DropIndexRequest) ( func (s *Server) ValidateAnalyzer(ctx context.Context, req *querypb.ValidateAnalyzerRequest) (*commonpb.Status, error) { return s.querynode.ValidateAnalyzer(ctx, req) } + +func (s *Server) GetHighlight(ctx context.Context, req *querypb.GetHighlightRequest) (*querypb.GetHighlightResponse, error) { + return s.querynode.GetHighlight(ctx, req) +} diff --git a/internal/distributed/querynode/service_test.go b/internal/distributed/querynode/service_test.go index 6c9cdd2a79..3546b18e4e 100644 --- a/internal/distributed/querynode/service_test.go +++ b/internal/distributed/querynode/service_test.go @@ -290,6 +290,18 @@ func Test_NewServer(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) }) + t.Run("GetHighlight", func(t *testing.T) { + mockQN.EXPECT().GetHighlight(mock.Anything, mock.Anything).Return(&querypb.GetHighlightResponse{ + Status: merr.Success(), + }, nil) + + resp, err := server.GetHighlight(ctx, &querypb.GetHighlightRequest{ + Channel: "test-channel", + }) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) + }) + t.Run("ValidateAnalyzer", func(t *testing.T) { mockQN.EXPECT().ValidateAnalyzer(mock.Anything, mock.Anything).Return(&commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil) req := &querypb.ValidateAnalyzerRequest{} diff --git a/internal/mocks/mock_querynode.go b/internal/mocks/mock_querynode.go index de357a3147..ba6d9016c1 100644 --- a/internal/mocks/mock_querynode.go +++ b/internal/mocks/mock_querynode.go @@ -370,6 +370,65 @@ func (_c *MockQueryNode_GetDataDistribution_Call) RunAndReturn(run func(context. return _c } +// GetHighlight provides a mock function with given fields: _a0, _a1 +func (_m *MockQueryNode) GetHighlight(_a0 context.Context, _a1 *querypb.GetHighlightRequest) (*querypb.GetHighlightResponse, error) { + ret := _m.Called(_a0, _a1) + + if len(ret) == 0 { + panic("no return value specified for GetHighlight") + } + + var r0 *querypb.GetHighlightResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *querypb.GetHighlightRequest) (*querypb.GetHighlightResponse, error)); ok { + return rf(_a0, _a1) + } + if rf, ok := ret.Get(0).(func(context.Context, *querypb.GetHighlightRequest) *querypb.GetHighlightResponse); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*querypb.GetHighlightResponse) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *querypb.GetHighlightRequest) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockQueryNode_GetHighlight_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetHighlight' +type MockQueryNode_GetHighlight_Call struct { + *mock.Call +} + +// GetHighlight is a helper method to define mock.On call +// - _a0 context.Context +// - _a1 *querypb.GetHighlightRequest +func (_e *MockQueryNode_Expecter) GetHighlight(_a0 interface{}, _a1 interface{}) *MockQueryNode_GetHighlight_Call { + return &MockQueryNode_GetHighlight_Call{Call: _e.mock.On("GetHighlight", _a0, _a1)} +} + +func (_c *MockQueryNode_GetHighlight_Call) Run(run func(_a0 context.Context, _a1 *querypb.GetHighlightRequest)) *MockQueryNode_GetHighlight_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*querypb.GetHighlightRequest)) + }) + return _c +} + +func (_c *MockQueryNode_GetHighlight_Call) Return(_a0 *querypb.GetHighlightResponse, _a1 error) *MockQueryNode_GetHighlight_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockQueryNode_GetHighlight_Call) RunAndReturn(run func(context.Context, *querypb.GetHighlightRequest) (*querypb.GetHighlightResponse, error)) *MockQueryNode_GetHighlight_Call { + _c.Call.Return(run) + return _c +} + // GetMetrics provides a mock function with given fields: _a0, _a1 func (_m *MockQueryNode) GetMetrics(_a0 context.Context, _a1 *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { ret := _m.Called(_a0, _a1) diff --git a/internal/mocks/mock_querynode_client.go b/internal/mocks/mock_querynode_client.go index e6bcf0ece6..f7054b94ba 100644 --- a/internal/mocks/mock_querynode_client.go +++ b/internal/mocks/mock_querynode_client.go @@ -446,6 +446,80 @@ func (_c *MockQueryNodeClient_GetDataDistribution_Call) RunAndReturn(run func(co return _c } +// GetHighlight provides a mock function with given fields: ctx, in, opts +func (_m *MockQueryNodeClient) GetHighlight(ctx context.Context, in *querypb.GetHighlightRequest, opts ...grpc.CallOption) (*querypb.GetHighlightResponse, error) { + _va := make([]interface{}, len(opts)) + for _i := range opts { + _va[_i] = opts[_i] + } + var _ca []interface{} + _ca = append(_ca, ctx, in) + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + if len(ret) == 0 { + panic("no return value specified for GetHighlight") + } + + var r0 *querypb.GetHighlightResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *querypb.GetHighlightRequest, ...grpc.CallOption) (*querypb.GetHighlightResponse, error)); ok { + return rf(ctx, in, opts...) + } + if rf, ok := ret.Get(0).(func(context.Context, *querypb.GetHighlightRequest, ...grpc.CallOption) *querypb.GetHighlightResponse); ok { + r0 = rf(ctx, in, opts...) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*querypb.GetHighlightResponse) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *querypb.GetHighlightRequest, ...grpc.CallOption) error); ok { + r1 = rf(ctx, in, opts...) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockQueryNodeClient_GetHighlight_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetHighlight' +type MockQueryNodeClient_GetHighlight_Call struct { + *mock.Call +} + +// GetHighlight is a helper method to define mock.On call +// - ctx context.Context +// - in *querypb.GetHighlightRequest +// - opts ...grpc.CallOption +func (_e *MockQueryNodeClient_Expecter) GetHighlight(ctx interface{}, in interface{}, opts ...interface{}) *MockQueryNodeClient_GetHighlight_Call { + return &MockQueryNodeClient_GetHighlight_Call{Call: _e.mock.On("GetHighlight", + append([]interface{}{ctx, in}, opts...)...)} +} + +func (_c *MockQueryNodeClient_GetHighlight_Call) Run(run func(ctx context.Context, in *querypb.GetHighlightRequest, opts ...grpc.CallOption)) *MockQueryNodeClient_GetHighlight_Call { + _c.Call.Run(func(args mock.Arguments) { + variadicArgs := make([]grpc.CallOption, len(args)-2) + for i, a := range args[2:] { + if a != nil { + variadicArgs[i] = a.(grpc.CallOption) + } + } + run(args[0].(context.Context), args[1].(*querypb.GetHighlightRequest), variadicArgs...) + }) + return _c +} + +func (_c *MockQueryNodeClient_GetHighlight_Call) Return(_a0 *querypb.GetHighlightResponse, _a1 error) *MockQueryNodeClient_GetHighlight_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockQueryNodeClient_GetHighlight_Call) RunAndReturn(run func(context.Context, *querypb.GetHighlightRequest, ...grpc.CallOption) (*querypb.GetHighlightResponse, error)) *MockQueryNodeClient_GetHighlight_Call { + _c.Call.Return(run) + return _c +} + // GetMetrics provides a mock function with given fields: ctx, in, opts func (_m *MockQueryNodeClient) GetMetrics(ctx context.Context, in *milvuspb.GetMetricsRequest, opts ...grpc.CallOption) (*milvuspb.GetMetricsResponse, error) { _va := make([]interface{}, len(opts)) diff --git a/internal/proxy/search_pipeline.go b/internal/proxy/search_pipeline.go index cb3c1b6124..31fede414c 100644 --- a/internal/proxy/search_pipeline.go +++ b/internal/proxy/search_pipeline.go @@ -21,6 +21,7 @@ import ( "context" "fmt" + "github.com/cockroachdb/errors" "github.com/samber/lo" "go.opentelemetry.io/otel" "go.opentelemetry.io/otel/trace" @@ -29,13 +30,16 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/parser/planparserv2" + "github.com/milvus-io/milvus/internal/proxy/shardclient" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/function/rerank" "github.com/milvus-io/milvus/internal/util/segcore" "github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "github.com/milvus-io/milvus/pkg/v2/proto/planpb" + "github.com/milvus-io/milvus/pkg/v2/proto/querypb" "github.com/milvus-io/milvus/pkg/v2/util/commonpbutil" "github.com/milvus-io/milvus/pkg/v2/util/merr" "github.com/milvus-io/milvus/pkg/v2/util/paramtable" @@ -112,6 +116,7 @@ const ( organizeOp = "organize" filterFieldOp = "filter_field" lambdaOp = "lambda" + highlightOp = "highlight" ) var opFactory = map[string]func(t *searchTask, params map[string]any) (operator, error){ @@ -122,6 +127,7 @@ var opFactory = map[string]func(t *searchTask, params map[string]any) (operator, requeryOp: newRequeryOperator, lambdaOp: newLambdaOperator, filterFieldOp: newFilterFieldOperator, + highlightOp: newHighlightOperator, } func NewNode(info *nodeDef, t *searchTask) (*Node, error) { @@ -592,6 +598,154 @@ func (op *filterFieldOperator) run(ctx context.Context, span trace.Span, inputs return []any{result}, nil } +const ( + PreTagsKey = "pre_tags" + PostTagsKey = "post_tags" + HighlightSearchTextKey = "highlight_search_data" + DefaultPreTag = "" + DefaultPostTag = "" +) + +type highlightTask struct { + *querypb.HighlightTask + preTags [][]byte + postTags [][]byte +} + +type highlightOperator struct { + tasks []*highlightTask + fieldSchemas []*schemapb.FieldSchema + lbPolicy shardclient.LBPolicy + scheduler *taskScheduler + + collectionName string + collectionID int64 + dbName string + + preTag []byte + postTag []byte +} + +func newHighlightOperator(t *searchTask, _ map[string]any) (operator, error) { + return &highlightOperator{ + tasks: t.highlightTasks, + lbPolicy: t.lb, + scheduler: t.node.(*Proxy).sched, + fieldSchemas: typeutil.GetAllFieldSchemas(t.schema.CollectionSchema), + collectionName: t.request.CollectionName, + collectionID: t.CollectionID, + dbName: t.request.DbName, + }, nil +} + +func (op *highlightOperator) run(ctx context.Context, span trace.Span, inputs ...any) ([]any, error) { + result := inputs[0].(*milvuspb.SearchResults) + datas := result.Results.GetFieldsData() + req := &querypb.GetHighlightRequest{ + Topks: result.GetResults().GetTopks(), + Tasks: lo.Map(op.tasks, func(task *highlightTask, _ int) *querypb.HighlightTask { return task.HighlightTask }), + } + + for _, task := range req.GetTasks() { + textFieldDatas, ok := lo.Find(datas, func(data *schemapb.FieldData) bool { return data.FieldId == task.GetFieldId() }) + if !ok { + return nil, errors.Errorf("get highlight failed, text field not in output field %s: %d", task.GetFieldName(), task.GetFieldId()) + } + texts := textFieldDatas.GetScalars().GetStringData().GetData() + task.Texts = append(task.Texts, texts...) + task.CorpusTextNum = int64(len(texts)) + field, ok := lo.Find(op.fieldSchemas, func(schema *schemapb.FieldSchema) bool { + return schema.GetFieldID() == task.GetFieldId() + }) + if !ok { + return nil, errors.Errorf("get highlight failed, field not found in schema %s: %d", task.GetFieldName(), task.GetFieldId()) + } + + // if use multi analyzer + // get analyzer field data + helper := typeutil.CreateFieldSchemaHelper(field) + if v, ok := helper.GetMultiAnalyzerParams(); ok { + params := map[string]any{} + err := json.Unmarshal([]byte(v), ¶ms) + if err != nil { + return nil, errors.Errorf("get highlight failed, get invalid multi analyzer params-: %v", err) + } + analyzerField, ok := params["by_field"] + if !ok { + return nil, errors.Errorf("get highlight failed, get invalid multi analyzer params, no by_field") + } + + analyzerFieldDatas, ok := lo.Find(datas, func(data *schemapb.FieldData) bool { return data.FieldName == analyzerField.(string) }) + if !ok { + return nil, errors.Errorf("get highlight failed, analyzer field not in output field") + } + task.AnalyzerNames = append(task.AnalyzerNames, analyzerFieldDatas.GetScalars().GetStringData().GetData()...) + } + } + + task := &HighlightTask{ + ctx: ctx, + lb: op.lbPolicy, + Condition: NewTaskCondition(ctx), + GetHighlightRequest: req, + collectionName: op.collectionName, + collectionID: op.collectionID, + dbName: op.dbName, + } + if err := op.scheduler.dqQueue.Enqueue(task); err != nil { + return nil, err + } + + if err := task.WaitToFinish(); err != nil { + return nil, err + } + + rowNum := len(result.Results.GetScores()) + HighlightResults := []*commonpb.HighlightResult{} + if rowNum != 0 { + rowDatas := lo.Map(task.result.Results, func(result *querypb.HighlightResult, i int) *commonpb.HighlightData { + return buildStringFragments(op.tasks[i/rowNum], i%rowNum, result.GetFragments()) + }) + + for i, task := range req.GetTasks() { + HighlightResults = append(HighlightResults, &commonpb.HighlightResult{ + FieldName: task.GetFieldName(), + Datas: rowDatas[i*rowNum : (i+1)*rowNum], + }) + } + } + + result.Results.HighlightResults = HighlightResults + return []any{result}, nil +} + +func buildStringFragments(task *highlightTask, idx int, frags []*querypb.HighlightFragment) *commonpb.HighlightData { + bytes := []byte(task.Texts[int(task.GetSearchTextNum())+idx]) + preTagsNum := len(task.preTags) + postTagsNum := len(task.postTags) + result := &commonpb.HighlightData{Fragments: make([]string, 0)} + for _, frag := range frags { + fragBytes := []byte{} + cursor := int(frag.GetStartOffset()) + for i := 0; i < len(frag.GetOffsets())/2; i++ { + startOffset := int(frag.Offsets[i<<1]) + endOffset := int(frag.Offsets[(i<<1)+1]) + if cursor < startOffset { + fragBytes = append(fragBytes, bytes[cursor:startOffset]...) + } + fragBytes = append(fragBytes, task.preTags[i%preTagsNum]...) + fragBytes = append(fragBytes, bytes[startOffset:endOffset]...) + fragBytes = append(fragBytes, task.postTags[i%postTagsNum]...) + cursor = endOffset + } + if cursor < int(frag.GetEndOffset()) { + fragBytes = append(fragBytes, bytes[cursor:frag.GetEndOffset()]...) + } + result.Fragments = append(result.Fragments, string(fragBytes)) + } + return result +} + func mergeIDsFunc(ctx context.Context, span trace.Span, inputs ...any) ([]any, error) { multipleMilvusResults := inputs[0].([]*milvuspb.SearchResults) idInt64Type := false @@ -648,6 +802,17 @@ func newPipeline(pipeDef *pipelineDef, t *searchTask) (*pipeline, error) { return &pipeline{name: pipeDef.name, nodes: nodes}, nil } +func (p *pipeline) AddNodes(t *searchTask, nodes ...*nodeDef) error { + for _, def := range nodes { + node, err := NewNode(def, t) + if err != nil { + return err + } + p.nodes = append(p.nodes, node) + } + return nil +} + func (p *pipeline) Run(ctx context.Context, span trace.Span, toReduceResults []*internalpb.SearchResults, storageCost segcore.StorageCost) (*milvuspb.SearchResults, segcore.StorageCost, error) { log.Ctx(ctx).Debug("SearchPipeline run", zap.String("pipeline", p.String())) msg := opMsg{} @@ -678,6 +843,20 @@ type pipelineDef struct { nodes []*nodeDef } +var filterFieldNode = &nodeDef{ + name: "filter_field", + inputs: []string{"result"}, + outputs: []string{"output"}, + opName: filterFieldOp, +} + +var highlightNode = &nodeDef{ + name: "highlight", + inputs: []string{"result"}, + outputs: []string{"output"}, + opName: highlightOp, +} + var searchPipe = &pipelineDef{ name: "search", nodes: []*nodeDef{ @@ -699,12 +878,6 @@ var searchPipe = &pipelineDef{ }, opName: lambdaOp, }, - { - name: "filter_field", - inputs: []string{"result"}, - outputs: []string{"output"}, - opName: filterFieldOp, - }, }, } @@ -763,12 +936,6 @@ var searchWithRequeryPipe = &pipelineDef{ }, opName: lambdaOp, }, - { - name: "filter_field", - inputs: []string{"result"}, - outputs: []string{"output"}, - opName: filterFieldOp, - }, }, } @@ -821,12 +988,6 @@ var searchWithRerankPipe = &pipelineDef{ }, opName: lambdaOp, }, - { - name: "filter_field", - inputs: []string{"result"}, - outputs: []string{"output"}, - opName: filterFieldOp, - }, }, } @@ -895,12 +1056,6 @@ var searchWithRerankRequeryPipe = &pipelineDef{ }, opName: lambdaOp, }, - { - name: "filter_field", - inputs: []string{"result"}, - outputs: []string{"output"}, - opName: filterFieldOp, - }, }, } @@ -919,12 +1074,6 @@ var hybridSearchPipe = &pipelineDef{ outputs: []string{"result"}, opName: rerankOp, }, - { - name: "filter_field", - inputs: []string{"result"}, - outputs: []string{"output"}, - opName: filterFieldOp, - }, }, } @@ -1026,12 +1175,6 @@ var hybridSearchWithRequeryAndRerankByFieldDataPipe = &pipelineDef{ }, opName: lambdaOp, }, - { - name: "filter_field", - inputs: []string{"result"}, - outputs: []string{"output"}, - opName: filterFieldOp, - }, }, } @@ -1128,3 +1271,23 @@ func newBuiltInPipeline(t *searchTask) (*pipeline, error) { } return nil, fmt.Errorf("Unsupported pipeline") } + +func newSearchPipeline(t *searchTask) (*pipeline, error) { + p, err := newBuiltInPipeline(t) + if err != nil { + return nil, err + } + + if len(t.highlightTasks) > 0 { + err := p.AddNodes(t, highlightNode, filterFieldNode) + if err != nil { + return nil, err + } + } else { + err := p.AddNodes(t, filterFieldNode) + if err != nil { + return nil, err + } + } + return p, nil +} diff --git a/internal/proxy/search_pipeline_test.go b/internal/proxy/search_pipeline_test.go index 03838a71e9..0f3aaa5d28 100644 --- a/internal/proxy/search_pipeline_test.go +++ b/internal/proxy/search_pipeline_test.go @@ -24,6 +24,7 @@ import ( "time" "github.com/bytedance/mockey" + "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" "go.opentelemetry.io/otel" "go.opentelemetry.io/otel/trace" @@ -31,11 +32,15 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/mocks" + "github.com/milvus-io/milvus/internal/proxy/shardclient" "github.com/milvus-io/milvus/internal/util/function/models" "github.com/milvus-io/milvus/internal/util/function/rerank" "github.com/milvus-io/milvus/internal/util/segcore" "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "github.com/milvus-io/milvus/pkg/v2/proto/planpb" + "github.com/milvus-io/milvus/pkg/v2/proto/querypb" + "github.com/milvus-io/milvus/pkg/v2/util/merr" "github.com/milvus-io/milvus/pkg/v2/util/testutils" "github.com/milvus-io/milvus/pkg/v2/util/timerecord" ) @@ -267,6 +272,94 @@ func (s *SearchPipelineSuite) TestOrganizeOp() { fmt.Println(ret) } +func (s *SearchPipelineSuite) TestHighlightOp() { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + proxy := &Proxy{} + proxy.tsoAllocator = ×tampAllocator{ + tso: newMockTimestampAllocatorInterface(), + } + sched, err := newTaskScheduler(ctx, proxy.tsoAllocator) + s.Require().NoError(err) + + err = sched.Start() + s.Require().NoError(err) + defer sched.Close() + proxy.sched = sched + + collName := "test_coll_highlight" + fieldName2Types := map[string]schemapb.DataType{ + testVarCharField: schemapb.DataType_VarChar, + } + schema := constructCollectionSchemaByDataType(collName, fieldName2Types, testVarCharField, false) + + req := &milvuspb.SearchRequest{ + CollectionName: collName, + DbName: "default", + } + + highlightTasks := []*highlightTask{ + { + HighlightTask: &querypb.HighlightTask{ + Texts: []string{"target text"}, + FieldName: testVarCharField, + FieldId: 100, + }, + preTags: [][]byte{[]byte(DefaultPreTag)}, + postTags: [][]byte{[]byte(DefaultPostTag)}, + }, + } + + mockLb := shardclient.NewMockLBPolicy(s.T()) + searchTask := &searchTask{ + node: proxy, + highlightTasks: highlightTasks, + lb: mockLb, + schema: newSchemaInfo(schema), + request: req, + collectionName: collName, + SearchRequest: &internalpb.SearchRequest{ + CollectionID: 0, + }, + } + + op, err := opFactory[highlightOp](searchTask, map[string]any{}) + s.Require().NoError(err) + + // mockery + mockLb.EXPECT().ExecuteOneChannel(mock.Anything, mock.Anything).Run(func(ctx context.Context, workload shardclient.CollectionWorkLoad) { + qn := mocks.NewMockQueryNodeClient(s.T()) + qn.EXPECT().GetHighlight(mock.Anything, mock.Anything).Return( + &querypb.GetHighlightResponse{ + Status: merr.Success(), + Results: []*querypb.HighlightResult{}, + }, nil) + workload.Exec(ctx, 0, qn, "test_chan") + }).Return(nil) + + _, err = op.run(ctx, s.span, &milvuspb.SearchResults{ + Results: &schemapb.SearchResultData{ + TopK: 3, + Topks: []int64{1}, + FieldsData: []*schemapb.FieldData{{ + FieldName: testVarCharField, + FieldId: 100, + Field: &schemapb.FieldData_Scalars{ + Scalars: &schemapb.ScalarField{ + Data: &schemapb.ScalarField_StringData{ + StringData: &schemapb.StringArray{ + Data: []string{"match text"}, + }, + }, + }, + }, + }}, + }, + }) + s.NoError(err) +} + func (s *SearchPipelineSuite) TestSearchPipeline() { collectionName := "test" task := &searchTask{ @@ -296,8 +389,11 @@ func (s *SearchPipelineSuite) TestSearchPipeline() { queryInfos: []*planpb.QueryInfo{{}}, translatedOutputFields: []string{"intField"}, } + pipeline, err := newPipeline(searchPipe, task) s.NoError(err) + pipeline.AddNodes(task, filterFieldNode) + sr := genTestSearchResultData(2, 10, schemapb.DataType_Int64, "intField", 101, false) results, storageCost, err := pipeline.Run(context.Background(), s.span, []*internalpb.SearchResults{sr}, segcore.StorageCost{ScannedRemoteBytes: 100, ScannedTotalBytes: 250}) s.NoError(err) @@ -364,6 +460,8 @@ func (s *SearchPipelineSuite) TestSearchPipelineWithRequery() { pipeline, err := newPipeline(searchWithRequeryPipe, task) s.NoError(err) + pipeline.AddNodes(task, filterFieldNode) + results, storageCost, err := pipeline.Run(context.Background(), s.span, []*internalpb.SearchResults{ genTestSearchResultData(2, 10, schemapb.DataType_Int64, "intField", 101, false), }, segcore.StorageCost{ScannedRemoteBytes: 100, ScannedTotalBytes: 200}) @@ -435,6 +533,7 @@ func (s *SearchPipelineSuite) TestSearchWithRerankPipe() { pipeline, err := newPipeline(searchWithRerankPipe, task) s.NoError(err) + pipeline.AddNodes(task, filterFieldNode) searchResults := genTestSearchResultData(2, 10, schemapb.DataType_Int64, "intField", 101, false) results, _, err := pipeline.Run(context.Background(), s.span, []*internalpb.SearchResults{searchResults}, segcore.StorageCost{}) @@ -518,6 +617,7 @@ func (s *SearchPipelineSuite) TestSearchWithRerankRequeryPipe() { pipeline, err := newPipeline(searchWithRerankRequeryPipe, task) s.NoError(err) + pipeline.AddNodes(task, filterFieldNode) searchResults := genTestSearchResultData(2, 10, schemapb.DataType_Int64, "intField", 101, false) results, storageCost, err := pipeline.Run(context.Background(), s.span, []*internalpb.SearchResults{searchResults}, segcore.StorageCost{}) @@ -551,6 +651,7 @@ func (s *SearchPipelineSuite) TestHybridSearchPipe() { pipeline, err := newPipeline(hybridSearchPipe, task) s.NoError(err) + pipeline.AddNodes(task, filterFieldNode) f1 := genTestSearchResultData(2, 10, schemapb.DataType_Int64, "intField", 101, true) f2 := genTestSearchResultData(2, 10, schemapb.DataType_Int64, "intField", 101, true) @@ -663,6 +764,7 @@ func (s *SearchPipelineSuite) TestHybridSearchWithRequeryAndRerankByDataPipe() { pipeline, err := newPipeline(hybridSearchWithRequeryAndRerankByFieldDataPipe, task) s.NoError(err) + pipeline.AddNodes(task, filterFieldNode) d1 := genTestSearchResultData(2, 10, schemapb.DataType_Int64, "intField", 101, true) d2 := genTestSearchResultData(2, 10, schemapb.DataType_Int64, "intField", 101, true) @@ -704,6 +806,7 @@ func (s *SearchPipelineSuite) TestHybridSearchWithRequeryPipe() { pipeline, err := newPipeline(hybridSearchWithRequeryPipe, task) s.NoError(err) + pipeline.AddNodes(task, filterFieldNode) d1 := genTestSearchResultData(2, 10, schemapb.DataType_Int64, "intField", 101, true) d2 := genTestSearchResultData(2, 10, schemapb.DataType_Int64, "intField", 101, true) diff --git a/internal/proxy/task.go b/internal/proxy/task.go index 847ec12a75..9618cb5d66 100644 --- a/internal/proxy/task.go +++ b/internal/proxy/task.go @@ -64,6 +64,7 @@ const ( StrictCastKey = "strict_cast" RankGroupScorer = "rank_group_scorer" AnnsFieldKey = "anns_field" + AnalyzerKey = "analyzer_name" TopKKey = "topk" NQKey = "nq" MetricTypeKey = common.MetricTypeKey @@ -116,6 +117,7 @@ const ( ListResourceGroupsTaskName = "ListResourceGroupsTask" DescribeResourceGroupTaskName = "DescribeResourceGroupTask" RunAnalyzerTaskName = "RunAnalyzer" + HighlightTaskName = "Highlight" CreateDatabaseTaskName = "CreateCollectionTask" DropDatabaseTaskName = "DropDatabaseTaskName" @@ -3146,6 +3148,95 @@ func (t *RunAnalyzerTask) PostExecute(ctx context.Context) error { return nil } +// git highlight after search +type HighlightTask struct { + baseTask + Condition + *querypb.GetHighlightRequest + ctx context.Context + collectionName string + collectionID typeutil.UniqueID + dbName string + lb shardclient.LBPolicy + + result *querypb.GetHighlightResponse +} + +func (t *HighlightTask) TraceCtx() context.Context { + return t.ctx +} + +func (t *HighlightTask) ID() UniqueID { + return t.Base.MsgID +} + +func (t *HighlightTask) SetID(uid UniqueID) { + t.Base.MsgID = uid +} + +func (t *HighlightTask) Name() string { + return HighlightTaskName +} + +func (t *HighlightTask) Type() commonpb.MsgType { + return t.Base.MsgType +} + +func (t *HighlightTask) BeginTs() Timestamp { + return t.Base.Timestamp +} + +func (t *HighlightTask) EndTs() Timestamp { + return t.Base.Timestamp +} + +func (t *HighlightTask) SetTs(ts Timestamp) { + t.Base.Timestamp = ts +} + +func (t *HighlightTask) OnEnqueue() error { + if t.Base == nil { + t.Base = commonpbutil.NewMsgBase() + } + t.Base.MsgType = commonpb.MsgType_RunAnalyzer + t.Base.SourceID = paramtable.GetNodeID() + return nil +} + +func (t *HighlightTask) PreExecute(ctx context.Context) error { + return nil +} + +func (t *HighlightTask) getHighlightOnShardleader(ctx context.Context, nodeID int64, qn types.QueryNodeClient, channel string) error { + t.GetHighlightRequest.Channel = channel + resp, err := qn.GetHighlight(ctx, t.GetHighlightRequest) + if err != nil { + return err + } + + if err := merr.Error(resp.GetStatus()); err != nil { + return err + } + t.result = resp + return nil +} + +func (t *HighlightTask) Execute(ctx context.Context) error { + err := t.lb.ExecuteOneChannel(ctx, shardclient.CollectionWorkLoad{ + Db: t.dbName, + CollectionName: t.collectionName, + CollectionID: t.collectionID, + Nq: int64(len(t.GetTopks()) * len(t.GetTasks())), + Exec: t.getHighlightOnShardleader, + }) + + return err +} + +func (t *HighlightTask) PostExecute(ctx context.Context) error { + return nil +} + // isIgnoreGrowing is used to check if the request should ignore growing func isIgnoreGrowing(params []*commonpb.KeyValuePair) (bool, error) { for _, kv := range params { diff --git a/internal/proxy/task_search.go b/internal/proxy/task_search.go index 9897988038..0f61cd3312 100644 --- a/internal/proxy/task_search.go +++ b/internal/proxy/task_search.go @@ -17,6 +17,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/parser/planparserv2" "github.com/milvus-io/milvus/internal/proxy/accesslog" "github.com/milvus-io/milvus/internal/proxy/shardclient" @@ -80,8 +81,8 @@ type searchTask struct { translatedOutputFields []string userOutputFields []string userDynamicFields []string - - resultBuf *typeutil.ConcurrentSet[*internalpb.SearchResults] + highlightTasks []*highlightTask + resultBuf *typeutil.ConcurrentSet[*internalpb.SearchResults] partitionIDsSet *typeutil.ConcurrentSet[UniqueID] @@ -470,12 +471,16 @@ func (t *searchTask) initAdvancedSearchRequest(ctx context.Context) error { } // set analyzer name for sub search - analyzer, err := funcutil.GetAttrByKeyFromRepeatedKV("analyzer_name", subReq.GetSearchParams()) + analyzer, err := funcutil.GetAttrByKeyFromRepeatedKV(AnalyzerKey, subReq.GetSearchParams()) if err == nil { internalSubReq.AnalyzerName = analyzer } internalSubReq.FieldId = queryInfo.GetQueryFieldId() + if err := t.addHighlightTask(t.request.GetHighlighter(), internalSubReq.GetMetricType(), internalSubReq.FieldId, subReq.GetPlaceholderGroup(), internalSubReq.GetAnalyzerName()); err != nil { + return err + } + queryFieldIDs = append(queryFieldIDs, internalSubReq.FieldId) // set PartitionIDs for sub search if t.partitionKeyMode { @@ -562,6 +567,119 @@ func (t *searchTask) fillResult() { t.result.CollectionName = t.collectionName } +func (t *searchTask) getBM25SearchTexts(placeholder []byte) ([]string, error) { + pb := &commonpb.PlaceholderGroup{} + proto.Unmarshal(placeholder, pb) + + if len(pb.Placeholders) != 1 || len(pb.Placeholders[0].Values) == 0 { + return nil, merr.WrapErrParameterInvalidMsg("please provide varchar/text for BM25 Function based search") + } + + holder := pb.Placeholders[0] + if holder.Type != commonpb.PlaceholderType_VarChar { + return nil, merr.WrapErrParameterInvalidMsg(fmt.Sprintf("please provide varchar/text for BM25 Function based search, got %s", holder.Type.String())) + } + + texts := funcutil.GetVarCharFromPlaceholder(holder) + return texts, nil +} + +func (t *searchTask) createLexicalHighlighter(highlighter *commonpb.Highlighter, metricType string, annsField int64, placeholder []byte, analyzerName string) error { + task := &highlightTask{ + HighlightTask: &querypb.HighlightTask{}, + } + + params := funcutil.KeyValuePair2Map(highlighter.GetParams()) + + if metricType != metric.BM25 { + return merr.WrapErrParameterInvalidMsg(`Search highlight only support with metric type "BM25" but was: %s`, t.SearchRequest.GetMetricType()) + } + function, ok := getBM25FunctionOfAnnsField(annsField, t.schema.GetFunctions()) + if !ok { + return merr.WrapErrServiceInternal(`Search with highlight failed, input field of BM25 annsField not found`) + } + task.FieldId = function.InputFieldIds[0] + task.FieldName = function.InputFieldNames[0] + + if value, ok := params[HighlightSearchTextKey]; ok { + enable, err := strconv.ParseBool(value) + if err != nil { + return merr.WrapErrParameterInvalidMsg("unmarshal highlight_search_data as bool failed: %v", err) + } + + // now only support highlight with search + // so skip if highlight search not enable. + if !enable { + return nil + } + } + + // set pre_tags and post_tags + if value, ok := params[PreTagsKey]; ok { + tags := []string{} + if err := json.Unmarshal([]byte(value), &tags); err != nil { + return merr.WrapErrParameterInvalidMsg("unmarshal pre_tags as string list failed: %v", err) + } + if len(tags) == 0 { + return merr.WrapErrParameterInvalidMsg("pre_tags cannot be empty list") + } + + task.preTags = make([][]byte, len(tags)) + for i, tag := range tags { + task.preTags[i] = []byte(tag) + } + } else { + task.preTags = [][]byte{[]byte(DefaultPreTag)} + } + + if value, ok := params[PostTagsKey]; ok { + tags := []string{} + if err := json.Unmarshal([]byte(value), &tags); err != nil { + return merr.WrapErrParameterInvalidMsg("unmarshal post_tags as string list failed: %v", err) + } + if len(tags) == 0 { + return merr.WrapErrParameterInvalidMsg("post_tags cannot be empty list") + } + task.postTags = make([][]byte, len(tags)) + for i, tag := range tags { + task.postTags[i] = []byte(tag) + } + } else { + task.postTags = [][]byte{[]byte(DefaultPostTag)} + } + + // set bm25 search text as query texts + texts, err := t.getBM25SearchTexts(placeholder) + if err != nil { + return err + } + + task.Texts = texts + task.SearchTextNum = int64(len(texts)) + if analyzerName != "" { + task.AnalyzerNames = []string{} + for i := 0; i < len(texts); i++ { + task.AnalyzerNames = append(task.AnalyzerNames, analyzerName) + } + } + + t.highlightTasks = append(t.highlightTasks, task) + return nil +} + +func (t *searchTask) addHighlightTask(highlighter *commonpb.Highlighter, metricType string, annsField int64, placeholder []byte, analyzerName string) error { + if highlighter == nil { + return nil + } + + switch highlighter.GetType() { + case commonpb.HighlightType_Lexical: + return t.createLexicalHighlighter(highlighter, metricType, annsField, placeholder, analyzerName) + default: + return merr.WrapErrParameterInvalidMsg("unsupported highlight type: %v", highlighter.GetType()) + } +} + func (t *searchTask) initSearchRequest(ctx context.Context) error { ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "init search request") defer sp.End() @@ -639,11 +757,14 @@ func (t *searchTask) initSearchRequest(ctx context.Context) error { t.SearchRequest.GroupSize = queryInfo.GroupSize if t.SearchRequest.MetricType == metric.BM25 { - analyzer, err := funcutil.GetAttrByKeyFromRepeatedKV("analyzer_name", t.request.GetSearchParams()) + analyzer, err := funcutil.GetAttrByKeyFromRepeatedKV(AnalyzerKey, t.request.GetSearchParams()) if err == nil { t.SearchRequest.AnalyzerName = analyzer } } + if err := t.addHighlightTask(t.request.GetHighlighter(), t.SearchRequest.MetricType, t.SearchRequest.FieldId, t.request.GetPlaceholderGroup(), t.SearchRequest.GetAnalyzerName()); err != nil { + return err + } if embedding.HasNonBM25Functions(t.schema.CollectionSchema.Functions, []int64{queryInfo.GetQueryFieldId()}) { ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "Proxy-Search-call-function-udf") @@ -819,7 +940,7 @@ func (t *searchTask) PostExecute(ctx context.Context) error { t.isRecallEvaluation = isRecallEvaluation // call pipeline - pipeline, err := newBuiltInPipeline(t) + pipeline, err := newSearchPipeline(t) if err != nil { log.Warn("Faild to create post process pipeline") return err diff --git a/internal/proxy/task_search_test.go b/internal/proxy/task_search_test.go index ab02ae231a..ae4ffe3a15 100644 --- a/internal/proxy/task_search_test.go +++ b/internal/proxy/task_search_test.go @@ -4884,3 +4884,193 @@ func TestSearchTask_InitSearchRequestWithStructArrayFields(t *testing.T) { }) } } + +func TestSearchTask_AddHighlightTask(t *testing.T) { + paramtable.Init() + + // Create a schema with BM25 function + schema := &schemapb.CollectionSchema{ + Name: "test_highlight_collection", + Fields: []*schemapb.FieldSchema{ + { + FieldID: 100, + Name: "text_field", + DataType: schemapb.DataType_VarChar, + }, + { + FieldID: 101, + Name: "sparse_field", + DataType: schemapb.DataType_SparseFloatVector, + }, + }, + Functions: []*schemapb.FunctionSchema{ + { + Name: "bm25_func", + Type: schemapb.FunctionType_BM25, + InputFieldNames: []string{"text_field"}, + InputFieldIds: []int64{100}, + OutputFieldNames: []string{"sparse_field"}, + OutputFieldIds: []int64{101}, + }, + }, + } + + placeholder := &commonpb.PlaceholderGroup{ + Placeholders: []*commonpb.PlaceholderValue{{ + Type: commonpb.PlaceholderType_VarChar, + Values: [][]byte{[]byte("test_str")}, + }}, + } + + placeholderBytes, err := proto.Marshal(placeholder) + require.NoError(t, err) + + t.Run("lexical highlight success", func(t *testing.T) { + task := &searchTask{ + schema: &schemaInfo{ + CollectionSchema: schema, + }, + } + + highlighter := &commonpb.Highlighter{ + Type: commonpb.HighlightType_Lexical, + Params: []*commonpb.KeyValuePair{{Key: HighlightSearchTextKey, Value: "true"}}, + } + + err := task.addHighlightTask(highlighter, metric.BM25, 101, placeholderBytes, "") + assert.NoError(t, err) + assert.Equal(t, 1, len(task.highlightTasks)) + assert.Equal(t, int64(100), task.highlightTasks[0].FieldId) + assert.Equal(t, "text_field", task.highlightTasks[0].FieldName) + }) + + t.Run("Lexical highlight with custom tags", func(t *testing.T) { + task := &searchTask{ + schema: &schemaInfo{ + CollectionSchema: schema, + }, + } + + highlighter := &commonpb.Highlighter{ + Type: commonpb.HighlightType_Lexical, + Params: []*commonpb.KeyValuePair{{Key: HighlightSearchTextKey, Value: "true"}, {Key: "pre_tags", Value: `[""]`}, {Key: "post_tags", Value: `[""]`}}, + } + + err := task.addHighlightTask(highlighter, metric.BM25, 101, placeholderBytes, "") + assert.NoError(t, err) + assert.Equal(t, 1, len(task.highlightTasks)) + assert.Equal(t, 1, len(task.highlightTasks[0].preTags)) + assert.Equal(t, []byte(""), task.highlightTasks[0].preTags[0]) + assert.Equal(t, 1, len(task.highlightTasks[0].postTags)) + assert.Equal(t, []byte(""), task.highlightTasks[0].postTags[0]) + }) + + t.Run("lexical highlight with wrong metric type", func(t *testing.T) { + task := &searchTask{ + schema: &schemaInfo{ + CollectionSchema: schema, + }, + SearchRequest: &internalpb.SearchRequest{}, + request: &milvuspb.SearchRequest{}, + } + + highlighter := &commonpb.Highlighter{ + Type: commonpb.HighlightType_Lexical, + Params: []*commonpb.KeyValuePair{{Key: HighlightSearchTextKey, Value: "true"}}, + } + + err := task.addHighlightTask(highlighter, metric.L2, 101, placeholderBytes, "") + assert.Error(t, err) + }) + + t.Run("lexical highlight with invalid pre_tags type", func(t *testing.T) { + task := &searchTask{ + schema: &schemaInfo{ + CollectionSchema: schema, + }, + } + + highlighter := &commonpb.Highlighter{ + Type: commonpb.HighlightType_Lexical, + Params: []*commonpb.KeyValuePair{{Key: HighlightSearchTextKey, Value: "true"}, {Key: "pre_tags", Value: "not_a_list"}}, + } + + err := task.addHighlightTask(highlighter, metric.BM25, 101, placeholderBytes, "") + assert.Error(t, err) + }) + + t.Run("default lexical highlight but not BM25 field", func(t *testing.T) { + schemaWithoutBM25 := &schemapb.CollectionSchema{ + Name: "test_collection", + Fields: []*schemapb.FieldSchema{ + { + FieldID: 100, + Name: "vector_field", + DataType: schemapb.DataType_FloatVector, + }, + }, + } + + task := &searchTask{ + schema: &schemaInfo{ + CollectionSchema: schemaWithoutBM25, + }, + } + + highlighter := &commonpb.Highlighter{ + Type: commonpb.HighlightType_Lexical, + Params: []*commonpb.KeyValuePair{{Key: HighlightSearchTextKey, Value: "true"}}, + } + + err := task.addHighlightTask(highlighter, metric.BM25, 100, placeholderBytes, "") + assert.Error(t, err) + }) + + t.Run("highlight without highlight search text", func(t *testing.T) { + task := &searchTask{ + schema: &schemaInfo{ + CollectionSchema: schema, + }, + } + + highlighter := &commonpb.Highlighter{ + Type: commonpb.HighlightType_Lexical, + Params: []*commonpb.KeyValuePair{{Key: HighlightSearchTextKey, Value: "false"}}, + } + + err := task.addHighlightTask(highlighter, metric.BM25, 101, placeholderBytes, "") + assert.NoError(t, err) + }) + + t.Run("highlight with invalid highlight search key", func(t *testing.T) { + task := &searchTask{ + schema: &schemaInfo{ + CollectionSchema: schema, + }, + } + + highlighter := &commonpb.Highlighter{ + Type: commonpb.HighlightType_Lexical, + Params: []*commonpb.KeyValuePair{{Key: HighlightSearchTextKey, Value: "invalid"}}, + } + + err := task.addHighlightTask(highlighter, metric.BM25, 101, placeholderBytes, "") + assert.Error(t, err) + }) + + t.Run("highlight with unknown type", func(t *testing.T) { + task := &searchTask{ + schema: &schemaInfo{ + CollectionSchema: schema, + }, + } + + highlighter := &commonpb.Highlighter{ + Type: 4, + Params: []*commonpb.KeyValuePair{{Key: HighlightSearchTextKey, Value: "true"}}, + } + + err := task.addHighlightTask(highlighter, metric.BM25, 101, placeholderBytes, "") + assert.Error(t, err) + }) +} diff --git a/internal/proxy/task_test.go b/internal/proxy/task_test.go index d46ec1e6ac..9e6d72ef84 100644 --- a/internal/proxy/task_test.go +++ b/internal/proxy/task_test.go @@ -39,6 +39,7 @@ import ( "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/mocks" + "github.com/milvus-io/milvus/internal/proxy/shardclient" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/util/function/embedding" "github.com/milvus-io/milvus/pkg/v2/common" @@ -5497,3 +5498,237 @@ func TestAlterCollection_AllowInsertAutoID_AutoIDFalse(t *testing.T) { assert.Error(t, err) assert.Equal(t, merr.Code(merr.ErrParameterInvalid), merr.Code(err)) } + +func TestHighlightTask(t *testing.T) { + paramtable.Init() + ctx := context.Background() + + t.Run("basic methods", func(t *testing.T) { + task := &HighlightTask{ + ctx: ctx, + GetHighlightRequest: &querypb.GetHighlightRequest{ + Base: commonpbutil.NewMsgBase(), + }, + Condition: NewTaskCondition(ctx), + } + + t.Run("traceCtx", func(t *testing.T) { + traceCtx := task.TraceCtx() + assert.NotNil(t, traceCtx) + assert.Equal(t, ctx, traceCtx) + }) + + t.Run("id", func(t *testing.T) { + id := UniqueID(uniquegenerator.GetUniqueIntGeneratorIns().GetInt()) + task.SetID(id) + assert.Equal(t, id, task.ID()) + }) + + t.Run("name", func(t *testing.T) { + assert.Equal(t, HighlightTaskName, task.Name()) + }) + + t.Run("type", func(t *testing.T) { + assert.Equal(t, commonpb.MsgType_Undefined, task.Type()) + }) + + t.Run("ts", func(t *testing.T) { + ts := Timestamp(time.Now().UnixNano()) + task.SetTs(ts) + assert.Equal(t, ts, task.BeginTs()) + assert.Equal(t, ts, task.EndTs()) + }) + }) + + t.Run("OnEnqueue", func(t *testing.T) { + task := &HighlightTask{ + ctx: ctx, + GetHighlightRequest: &querypb.GetHighlightRequest{}, + Condition: NewTaskCondition(ctx), + } + + err := task.OnEnqueue() + assert.NoError(t, err) + assert.NotNil(t, task.Base) + assert.Equal(t, commonpb.MsgType_RunAnalyzer, task.Base.MsgType) + assert.Equal(t, paramtable.GetNodeID(), task.Base.SourceID) + }) + + t.Run("PreExecute", func(t *testing.T) { + task := &HighlightTask{ + ctx: ctx, + GetHighlightRequest: &querypb.GetHighlightRequest{ + Base: commonpbutil.NewMsgBase(), + }, + Condition: NewTaskCondition(ctx), + } + + err := task.PreExecute(ctx) + assert.NoError(t, err) + }) + + t.Run("getHighlightOnShardleader success", func(t *testing.T) { + task := &HighlightTask{ + ctx: ctx, + GetHighlightRequest: &querypb.GetHighlightRequest{ + Base: commonpbutil.NewMsgBase(), + Topks: []int64{10}, + Tasks: []*querypb.HighlightTask{ + {}, + }, + }, + Condition: NewTaskCondition(ctx), + } + + mockQN := mocks.NewMockQueryNodeClient(t) + expectedResp := &querypb.GetHighlightResponse{ + Status: merr.Success(), + Results: []*querypb.HighlightResult{ + { + Fragments: []*querypb.HighlightFragment{ + { + StartOffset: 0, + EndOffset: 10, + Offsets: []int64{0, 5, 5, 10}, + }, + }, + }, + }, + } + + mockQN.EXPECT().GetHighlight(mock.Anything, mock.Anything).Return(expectedResp, nil) + + err := task.getHighlightOnShardleader(ctx, 1, mockQN, "test_channel") + assert.NoError(t, err) + assert.NotNil(t, task.result) + assert.Equal(t, expectedResp, task.result) + assert.Equal(t, "test_channel", task.GetHighlightRequest.Channel) + }) + + t.Run("getHighlightOnShardleader rpc error", func(t *testing.T) { + task := &HighlightTask{ + ctx: ctx, + GetHighlightRequest: &querypb.GetHighlightRequest{ + Base: commonpbutil.NewMsgBase(), + }, + Condition: NewTaskCondition(ctx), + } + + mockQN := mocks.NewMockQueryNodeClient(t) + mockQN.EXPECT().GetHighlight(mock.Anything, mock.Anything).Return(nil, errors.New("rpc error")) + + err := task.getHighlightOnShardleader(ctx, 1, mockQN, "test_channel") + assert.Error(t, err) + assert.Contains(t, err.Error(), "rpc error") + }) + + t.Run("getHighlightOnShardleader status error", func(t *testing.T) { + task := &HighlightTask{ + ctx: ctx, + GetHighlightRequest: &querypb.GetHighlightRequest{ + Base: commonpbutil.NewMsgBase(), + }, + Condition: NewTaskCondition(ctx), + } + + mockQN := mocks.NewMockQueryNodeClient(t) + expectedResp := &querypb.GetHighlightResponse{ + Status: merr.Status(errors.New("status error")), + } + + mockQN.EXPECT().GetHighlight(mock.Anything, mock.Anything).Return(expectedResp, nil) + + err := task.getHighlightOnShardleader(ctx, 1, mockQN, "test_channel") + assert.Error(t, err) + }) + + t.Run("Execute success", func(t *testing.T) { + task := &HighlightTask{ + ctx: ctx, + collectionName: "test_collection", + collectionID: 100, + dbName: "default", + GetHighlightRequest: &querypb.GetHighlightRequest{ + Base: commonpbutil.NewMsgBase(), + Topks: []int64{10, 20}, + Tasks: []*querypb.HighlightTask{ + {}, + {}, + }, + }, + Condition: NewTaskCondition(ctx), + } + + mockLB := shardclient.NewMockLBPolicy(t) + mockQN := mocks.NewMockQueryNodeClient(t) + + expectedResp := &querypb.GetHighlightResponse{ + Status: merr.Success(), + Results: []*querypb.HighlightResult{ + { + Fragments: []*querypb.HighlightFragment{ + { + StartOffset: 0, + EndOffset: 10, + Offsets: []int64{0, 5, 5, 10}, + }, + }, + }, + }, + } + + mockQN.EXPECT().GetHighlight(mock.Anything, mock.Anything).Return(expectedResp, nil) + + mockLB.EXPECT().ExecuteOneChannel(mock.Anything, mock.Anything).Run( + func(ctx context.Context, workload shardclient.CollectionWorkLoad) { + assert.Equal(t, "default", workload.Db) + assert.Equal(t, "test_collection", workload.CollectionName) + assert.Equal(t, int64(100), workload.CollectionID) + assert.Equal(t, int64(4), workload.Nq) // len(topks) * len(tasks) = 2 * 2 = 4 + err := workload.Exec(ctx, 1, mockQN, "test_channel") + assert.NoError(t, err) + }, + ).Return(nil) + + task.lb = mockLB + + err := task.Execute(ctx) + assert.NoError(t, err) + assert.NotNil(t, task.result) + }) + + t.Run("Execute lb error", func(t *testing.T) { + task := &HighlightTask{ + ctx: ctx, + collectionName: "test_collection", + collectionID: 100, + dbName: "default", + GetHighlightRequest: &querypb.GetHighlightRequest{ + Base: commonpbutil.NewMsgBase(), + }, + Condition: NewTaskCondition(ctx), + } + + mockLB := shardclient.NewMockLBPolicy(t) + mockLB.EXPECT().ExecuteOneChannel(mock.Anything, mock.Anything).Return(errors.New("lb error")) + + task.lb = mockLB + + err := task.Execute(ctx) + assert.Error(t, err) + assert.Contains(t, err.Error(), "lb error") + }) + + t.Run("PostExecute", func(t *testing.T) { + task := &HighlightTask{ + ctx: ctx, + GetHighlightRequest: &querypb.GetHighlightRequest{ + Base: commonpbutil.NewMsgBase(), + }, + Condition: NewTaskCondition(ctx), + } + + err := task.PostExecute(ctx) + assert.NoError(t, err) + }) +} diff --git a/internal/proxy/util.go b/internal/proxy/util.go index 1713c69cb0..ae0698dec9 100644 --- a/internal/proxy/util.go +++ b/internal/proxy/util.go @@ -3025,3 +3025,9 @@ func genFunctionFields(ctx context.Context, insertMsg *msgstream.InsertMsg, sche } return nil } + +func getBM25FunctionOfAnnsField(fieldID int64, functions []*schemapb.FunctionSchema) (*schemapb.FunctionSchema, bool) { + return lo.Find(functions, func(function *schemapb.FunctionSchema) bool { + return function.GetType() == schemapb.FunctionType_BM25 && function.OutputFieldIds[0] == fieldID + }) +} diff --git a/internal/querycoordv2/mocks/mock_querynode.go b/internal/querycoordv2/mocks/mock_querynode.go index 6eea1f7802..7bd6e44f44 100644 --- a/internal/querycoordv2/mocks/mock_querynode.go +++ b/internal/querycoordv2/mocks/mock_querynode.go @@ -324,6 +324,65 @@ func (_c *MockQueryNodeServer_GetDataDistribution_Call) RunAndReturn(run func(co return _c } +// GetHighlight provides a mock function with given fields: _a0, _a1 +func (_m *MockQueryNodeServer) GetHighlight(_a0 context.Context, _a1 *querypb.GetHighlightRequest) (*querypb.GetHighlightResponse, error) { + ret := _m.Called(_a0, _a1) + + if len(ret) == 0 { + panic("no return value specified for GetHighlight") + } + + var r0 *querypb.GetHighlightResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *querypb.GetHighlightRequest) (*querypb.GetHighlightResponse, error)); ok { + return rf(_a0, _a1) + } + if rf, ok := ret.Get(0).(func(context.Context, *querypb.GetHighlightRequest) *querypb.GetHighlightResponse); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*querypb.GetHighlightResponse) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *querypb.GetHighlightRequest) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockQueryNodeServer_GetHighlight_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetHighlight' +type MockQueryNodeServer_GetHighlight_Call struct { + *mock.Call +} + +// GetHighlight is a helper method to define mock.On call +// - _a0 context.Context +// - _a1 *querypb.GetHighlightRequest +func (_e *MockQueryNodeServer_Expecter) GetHighlight(_a0 interface{}, _a1 interface{}) *MockQueryNodeServer_GetHighlight_Call { + return &MockQueryNodeServer_GetHighlight_Call{Call: _e.mock.On("GetHighlight", _a0, _a1)} +} + +func (_c *MockQueryNodeServer_GetHighlight_Call) Run(run func(_a0 context.Context, _a1 *querypb.GetHighlightRequest)) *MockQueryNodeServer_GetHighlight_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*querypb.GetHighlightRequest)) + }) + return _c +} + +func (_c *MockQueryNodeServer_GetHighlight_Call) Return(_a0 *querypb.GetHighlightResponse, _a1 error) *MockQueryNodeServer_GetHighlight_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockQueryNodeServer_GetHighlight_Call) RunAndReturn(run func(context.Context, *querypb.GetHighlightRequest) (*querypb.GetHighlightResponse, error)) *MockQueryNodeServer_GetHighlight_Call { + _c.Call.Return(run) + return _c +} + // GetMetrics provides a mock function with given fields: _a0, _a1 func (_m *MockQueryNodeServer) GetMetrics(_a0 context.Context, _a1 *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { ret := _m.Called(_a0, _a1) diff --git a/internal/querynodev2/delegator/delegator.go b/internal/querynodev2/delegator/delegator.go index 93833795df..8a86ec60f9 100644 --- a/internal/querynodev2/delegator/delegator.go +++ b/internal/querynodev2/delegator/delegator.go @@ -104,6 +104,7 @@ type ShardDelegator interface { // analyzer RunAnalyzer(ctx context.Context, req *querypb.RunAnalyzerRequest) ([]*milvuspb.AnalyzerResult, error) + GetHighlight(ctx context.Context, req *querypb.GetHighlightRequest) ([]*querypb.HighlightResult, error) // control Serviceable() bool diff --git a/internal/querynodev2/delegator/delegator_data.go b/internal/querynodev2/delegator/delegator_data.go index 9bc846e4a3..8114bac0a1 100644 --- a/internal/querynodev2/delegator/delegator_data.go +++ b/internal/querynodev2/delegator/delegator_data.go @@ -29,6 +29,7 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/querynodev2/cluster" @@ -1001,3 +1002,57 @@ func (sd *shardDelegator) DropIndex(ctx context.Context, req *querypb.DropIndexR } return nil } + +func (sd *shardDelegator) GetHighlight(ctx context.Context, req *querypb.GetHighlightRequest) ([]*querypb.HighlightResult, error) { + result := []*querypb.HighlightResult{} + for _, task := range req.GetTasks() { + if len(task.GetTexts()) != int(task.GetSearchTextNum()+task.GetCorpusTextNum()) { + return nil, errors.Errorf("package highlight texts error, num of texts not equal the expected num %d:%d", len(task.GetTexts()), task.GetSearchTextNum()+task.GetCorpusTextNum()) + } + analyzer, ok := sd.analyzerRunners[task.GetFieldId()] + if !ok { + return nil, merr.WrapErrParameterInvalidMsg("get highlight failed, the highlight field not found, %s:%d", task.GetFieldName(), task.GetFieldId()) + } + topks := req.GetTopks() + var results [][]*milvuspb.AnalyzerToken + var err error + + if len(analyzer.GetInputFields()) == 1 { + results, err = analyzer.BatchAnalyze(true, false, task.GetTexts()) + if err != nil { + return nil, err + } + } else if len(analyzer.GetInputFields()) == 2 { + // use analyzer names if analyzer need two input field + results, err = analyzer.BatchAnalyze(true, false, task.GetTexts(), task.GetAnalyzerNames()) + if err != nil { + return nil, err + } + } + + // analyze result of search text + searchResults := results[0:task.SearchTextNum] + // analyze result of corpus text + corpusResults := results[task.SearchTextNum:] + corpusIdx := 0 + for i, tokens := range searchResults { + tokenSet := typeutil.NewSet[string]() + for _, token := range tokens { + tokenSet.Insert(token.GetToken()) + } + + for j := 0; j < int(topks[i]); j++ { + offsets := []int64{} + for _, token := range corpusResults[corpusIdx] { + if tokenSet.Contain(token.GetToken()) { + offsets = append(offsets, token.GetStartOffset(), token.GetEndOffset()) + } + } + result = append(result, &querypb.HighlightResult{Fragments: []*querypb.HighlightFragment{{StartOffset: 0, EndOffset: int64(len(task.Texts[int(task.SearchTextNum)+corpusIdx])), Offsets: offsets}}}) + corpusIdx++ + } + } + } + + return result, nil +} diff --git a/internal/querynodev2/delegator/delegator_test.go b/internal/querynodev2/delegator/delegator_test.go index 6543dd2b75..124724d0fc 100644 --- a/internal/querynodev2/delegator/delegator_test.go +++ b/internal/querynodev2/delegator/delegator_test.go @@ -1596,6 +1596,159 @@ func (s *DelegatorSuite) TestRunAnalyzer() { }) } +func (s *DelegatorSuite) TestGetHighlight() { + ctx := context.Background() + s.TestCreateDelegatorWithFunction() + + s.Run("field analyzer not exist", func() { + _, err := s.delegator.GetHighlight(ctx, &querypb.GetHighlightRequest{ + Topks: []int64{1}, + Tasks: []*querypb.HighlightTask{ + { + FieldId: 999, // non-existent field + }, + }, + }) + s.Require().Error(err) + }) + + s.Run("normal highlight with single analyzer", func() { + s.manager.Collection.PutOrRef(s.collectionID, &schemapb.CollectionSchema{ + Fields: []*schemapb.FieldSchema{ + { + FieldID: 100, + Name: "text", + DataType: schemapb.DataType_VarChar, + TypeParams: []*commonpb.KeyValuePair{{Key: "analyzer_params", Value: "{}"}}, + }, + { + FieldID: 101, + Name: "sparse", + DataType: schemapb.DataType_SparseFloatVector, + }, + }, + Functions: []*schemapb.FunctionSchema{{ + Type: schemapb.FunctionType_BM25, + InputFieldNames: []string{"text"}, + InputFieldIds: []int64{100}, + OutputFieldNames: []string{"sparse"}, + OutputFieldIds: []int64{101}, + }}, + }, nil, &querypb.LoadMetaInfo{SchemaVersion: tsoutil.ComposeTSByTime(time.Now(), 0)}) + s.ResetDelegator() + + result, err := s.delegator.GetHighlight(ctx, &querypb.GetHighlightRequest{ + Topks: []int64{2}, + Tasks: []*querypb.HighlightTask{ + { + FieldId: 100, + Texts: []string{"test", "this is a test document", "another test case"}, + SearchTextNum: 1, + CorpusTextNum: 2, + }, + }, + }) + s.Require().NoError(err) + s.Require().Equal(2, len(result)) + // Check that we got highlight results + s.Require().NotNil(result[0].Fragments) + s.Require().NotNil(result[1].Fragments) + }) + + s.Run("highlight with multi analyzer", func() { + s.manager.Collection.PutOrRef(s.collectionID, &schemapb.CollectionSchema{ + Fields: []*schemapb.FieldSchema{ + { + FieldID: 100, + Name: "text", + DataType: schemapb.DataType_VarChar, + TypeParams: []*commonpb.KeyValuePair{{Key: "multi_analyzer_params", Value: `{ + "by_field": "analyzer", + "analyzers": { + "standard": {}, + "default": {} + } + }`}}, + }, + { + FieldID: 101, + Name: "sparse", + DataType: schemapb.DataType_SparseFloatVector, + }, + { + FieldID: 102, + Name: "analyzer", + DataType: schemapb.DataType_VarChar, + }, + }, + Functions: []*schemapb.FunctionSchema{{ + Type: schemapb.FunctionType_BM25, + InputFieldNames: []string{"text"}, + InputFieldIds: []int64{100}, + OutputFieldNames: []string{"sparse"}, + OutputFieldIds: []int64{101}, + }}, + }, nil, &querypb.LoadMetaInfo{SchemaVersion: tsoutil.ComposeTSByTime(time.Now(), 0)}) + s.ResetDelegator() + + // two target with two analyzer + result, err := s.delegator.GetHighlight(ctx, &querypb.GetHighlightRequest{ + Topks: []int64{1, 1}, + Tasks: []*querypb.HighlightTask{ + { + FieldId: 100, + Texts: []string{"test1", "test2", "this is a test1 document", "another test2 case"}, + AnalyzerNames: []string{"default", "standard", "default", "default"}, + SearchTextNum: 2, + CorpusTextNum: 2, + }, + }, + }) + s.Require().NoError(err) + s.Require().Equal(2, len(result)) + }) + + s.Run("empty target texts", func() { + s.manager.Collection.PutOrRef(s.collectionID, &schemapb.CollectionSchema{ + Fields: []*schemapb.FieldSchema{ + { + FieldID: 100, + Name: "text", + DataType: schemapb.DataType_VarChar, + TypeParams: []*commonpb.KeyValuePair{{Key: "analyzer_params", Value: "{}"}}, + }, + { + FieldID: 101, + Name: "sparse", + DataType: schemapb.DataType_SparseFloatVector, + }, + }, + Functions: []*schemapb.FunctionSchema{{ + Type: schemapb.FunctionType_BM25, + InputFieldNames: []string{"text"}, + InputFieldIds: []int64{100}, + OutputFieldNames: []string{"sparse"}, + OutputFieldIds: []int64{101}, + }}, + }, nil, &querypb.LoadMetaInfo{SchemaVersion: tsoutil.ComposeTSByTime(time.Now(), 0)}) + s.ResetDelegator() + + result, err := s.delegator.GetHighlight(ctx, &querypb.GetHighlightRequest{ + Topks: []int64{1}, + Tasks: []*querypb.HighlightTask{ + { + FieldId: 100, + Texts: []string{"test document"}, + SearchTextNum: 0, + CorpusTextNum: 1, + }, + }, + }) + s.Require().NoError(err) + s.Require().NotNil(result) + }) +} + // TestDelegatorLifetimeIntegration tests the integration of lifetime state checks with main delegator methods func (s *DelegatorSuite) TestDelegatorLifetimeIntegration() { sd := s.delegator.(*shardDelegator) diff --git a/internal/querynodev2/delegator/mock_delegator.go b/internal/querynodev2/delegator/mock_delegator.go index 3d06d140e5..8f4ffeba45 100644 --- a/internal/querynodev2/delegator/mock_delegator.go +++ b/internal/querynodev2/delegator/mock_delegator.go @@ -289,6 +289,65 @@ func (_c *MockShardDelegator_GetDeleteBufferSize_Call) RunAndReturn(run func() ( return _c } +// GetHighlight provides a mock function with given fields: ctx, req +func (_m *MockShardDelegator) GetHighlight(ctx context.Context, req *querypb.GetHighlightRequest) ([]*querypb.HighlightResult, error) { + ret := _m.Called(ctx, req) + + if len(ret) == 0 { + panic("no return value specified for GetHighlight") + } + + var r0 []*querypb.HighlightResult + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *querypb.GetHighlightRequest) ([]*querypb.HighlightResult, error)); ok { + return rf(ctx, req) + } + if rf, ok := ret.Get(0).(func(context.Context, *querypb.GetHighlightRequest) []*querypb.HighlightResult); ok { + r0 = rf(ctx, req) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]*querypb.HighlightResult) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *querypb.GetHighlightRequest) error); ok { + r1 = rf(ctx, req) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockShardDelegator_GetHighlight_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetHighlight' +type MockShardDelegator_GetHighlight_Call struct { + *mock.Call +} + +// GetHighlight is a helper method to define mock.On call +// - ctx context.Context +// - req *querypb.GetHighlightRequest +func (_e *MockShardDelegator_Expecter) GetHighlight(ctx interface{}, req interface{}) *MockShardDelegator_GetHighlight_Call { + return &MockShardDelegator_GetHighlight_Call{Call: _e.mock.On("GetHighlight", ctx, req)} +} + +func (_c *MockShardDelegator_GetHighlight_Call) Run(run func(ctx context.Context, req *querypb.GetHighlightRequest)) *MockShardDelegator_GetHighlight_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*querypb.GetHighlightRequest)) + }) + return _c +} + +func (_c *MockShardDelegator_GetHighlight_Call) Return(_a0 []*querypb.HighlightResult, _a1 error) *MockShardDelegator_GetHighlight_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockShardDelegator_GetHighlight_Call) RunAndReturn(run func(context.Context, *querypb.GetHighlightRequest) ([]*querypb.HighlightResult, error)) *MockShardDelegator_GetHighlight_Call { + _c.Call.Return(run) + return _c +} + // GetPartitionStatsVersions provides a mock function with given fields: ctx func (_m *MockShardDelegator) GetPartitionStatsVersions(ctx context.Context) map[int64]int64 { ret := _m.Called(ctx) diff --git a/internal/querynodev2/services.go b/internal/querynodev2/services.go index 53ddd5ba4d..53ba81fa7e 100644 --- a/internal/querynodev2/services.go +++ b/internal/querynodev2/services.go @@ -1741,3 +1741,36 @@ func (node *QueryNode) DropIndex(ctx context.Context, req *querypb.DropIndexRequ return merr.Success(), nil } + +func (node *QueryNode) GetHighlight(ctx context.Context, req *querypb.GetHighlightRequest) (*querypb.GetHighlightResponse, error) { + // check node healthy + if err := node.lifetime.Add(merr.IsHealthy); err != nil { + return &querypb.GetHighlightResponse{ + Status: merr.Status(err), + }, nil + } + defer node.lifetime.Done() + + // get delegator + sd, ok := node.delegators.Get(req.GetChannel()) + if !ok { + err := merr.WrapErrChannelNotFound(req.GetChannel()) + log.Warn("GetHighlight failed, failed to get shard delegator", zap.Error(err)) + return &querypb.GetHighlightResponse{ + Status: merr.Status(err), + }, nil + } + + results, err := sd.GetHighlight(ctx, req) + if err != nil { + log.Warn("GetHighlight failed, delegator run failed", zap.Error(err)) + return &querypb.GetHighlightResponse{ + Status: merr.Status(err), + }, nil + } + + return &querypb.GetHighlightResponse{ + Status: merr.Success(), + Results: results, + }, nil +} diff --git a/internal/querynodev2/services_test.go b/internal/querynodev2/services_test.go index 98c8159b4f..95a0cb7347 100644 --- a/internal/querynodev2/services_test.go +++ b/internal/querynodev2/services_test.go @@ -2471,6 +2471,49 @@ func (suite *ServiceSuite) TestValidateAnalyzer() { }) } +func (suite *ServiceSuite) TestGetHighlight() { + ctx := context.Background() + + suite.Run("node not healthy", func() { + suite.node.UpdateStateCode(commonpb.StateCode_Abnormal) + defer suite.node.UpdateStateCode(commonpb.StateCode_Healthy) + + resp, err := suite.node.GetHighlight(ctx, &querypb.GetHighlightRequest{ + Channel: suite.vchannel, + Topks: []int64{10}, + }) + + suite.NoError(err) + suite.Error(merr.Error(resp.GetStatus())) + }) + + suite.Run("normal case", func() { + delegator := &delegator.MockShardDelegator{} + suite.node.delegators.Insert(suite.vchannel, delegator) + defer suite.node.delegators.GetAndRemove(suite.vchannel) + delegator.EXPECT().GetHighlight(mock.Anything, mock.Anything).Return( + []*querypb.HighlightResult{}, nil) + resp, err := suite.node.GetHighlight(ctx, &querypb.GetHighlightRequest{ + Channel: suite.vchannel, + Topks: []int64{1, 1}, + Tasks: []*querypb.HighlightTask{ + { + FieldName: "text_field", + FieldId: 100, + Texts: []string{"target text", "target text2", "text", "text2"}, + AnalyzerNames: []string{"standard", "standard", "standard", "standard"}, + SearchTextNum: 2, + CorpusTextNum: 2, + }, + }, + }) + + suite.NoError(err) + suite.NoError(merr.Error(resp.GetStatus())) + suite.NotNil(resp.Results) + }) +} + func TestQueryNodeService(t *testing.T) { wal := mock_streaming.NewMockWALAccesser(t) local := mock_streaming.NewMockLocal(t) diff --git a/internal/util/mock/grpc_querynode_client.go b/internal/util/mock/grpc_querynode_client.go index daf4df720a..22f7fd245d 100644 --- a/internal/util/mock/grpc_querynode_client.go +++ b/internal/util/mock/grpc_querynode_client.go @@ -146,6 +146,10 @@ func (m *GrpcQueryNodeClient) ValidateAnalyzer(ctx context.Context, in *querypb. return &commonpb.Status{}, m.Err } +func (m *GrpcQueryNodeClient) GetHighlight(ctx context.Context, in *querypb.GetHighlightRequest, opts ...grpc.CallOption) (*querypb.GetHighlightResponse, error) { + return &querypb.GetHighlightResponse{}, m.Err +} + func (m *GrpcQueryNodeClient) Close() error { return m.Err } diff --git a/internal/util/wrappers/qn_wrapper.go b/internal/util/wrappers/qn_wrapper.go index 5ac4bb6ab0..a48aacba27 100644 --- a/internal/util/wrappers/qn_wrapper.go +++ b/internal/util/wrappers/qn_wrapper.go @@ -160,6 +160,10 @@ func (qn *qnServerWrapper) RunAnalyzer(ctx context.Context, in *querypb.RunAnaly return qn.QueryNode.RunAnalyzer(ctx, in) } +func (qn *qnServerWrapper) GetHighlight(ctx context.Context, in *querypb.GetHighlightRequest, _ ...grpc.CallOption) (*querypb.GetHighlightResponse, error) { + return qn.QueryNode.GetHighlight(ctx, in) +} + func (qn *qnServerWrapper) DropIndex(ctx context.Context, in *querypb.DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { return qn.QueryNode.DropIndex(ctx, in) } diff --git a/pkg/proto/query_coord.proto b/pkg/proto/query_coord.proto index 05c1c9cf27..f2eb277cc0 100644 --- a/pkg/proto/query_coord.proto +++ b/pkg/proto/query_coord.proto @@ -174,6 +174,8 @@ service QueryNode { rpc UpdateSchema(UpdateSchemaRequest) returns (common.Status) {} rpc RunAnalyzer(RunAnalyzerRequest) returns(milvus.RunAnalyzerResponse){} + rpc GetHighlight(GetHighlightRequest) returns (GetHighlightResponse){} + rpc DropIndex(DropIndexRequest) returns (common.Status) {} rpc ValidateAnalyzer(ValidateAnalyzerRequest) returns(common.Status){} } @@ -1012,6 +1014,42 @@ message ValidateAnalyzerRequest{ repeated AnalyzerInfo analyzer_infos = 2; } +// HighlightTask fetch highlight for all queries at one field +// len(texts) == search_text_num + corpus_text_num +message HighlightTask{ + string field_name = 1; + int64 field_id = 2; + repeated string texts = 3; + repeated string analyzer_names = 4; // used if field with multi-analyzer + + int64 search_text_num = 5; + int64 corpus_text_num = 6; +} + +message GetHighlightRequest{ + common.MsgBase base = 1; + string channel = 2; + repeated int64 topks = 3; + repeated HighlightTask tasks=4; // one task for one field +} + +// start_offset and end_offset are fragment offset in the original text +// number of offsets always be 2 * number of highlight terms in the fragment +message HighlightFragment{ + int64 start_offset = 1; + int64 end_offset = 2; + repeated int64 offsets = 3; +} + +message HighlightResult{ + repeated HighlightFragment fragments = 2; +} + +message GetHighlightResponse{ + common.Status status = 1; + repeated HighlightResult results = 2; +} + message ListLoadedSegmentsRequest { common.MsgBase base = 1; } diff --git a/pkg/proto/querypb/query_coord.pb.go b/pkg/proto/querypb/query_coord.pb.go index 58ecbb74ff..18d05f750d 100644 --- a/pkg/proto/querypb/query_coord.pb.go +++ b/pkg/proto/querypb/query_coord.pb.go @@ -7763,6 +7763,333 @@ func (x *ValidateAnalyzerRequest) GetAnalyzerInfos() []*AnalyzerInfo { return nil } +// HighlightTask fetch highlight for all queries at one field +// len(texts) == search_text_num + corpus_text_num +type HighlightTask struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + FieldName string `protobuf:"bytes,1,opt,name=field_name,json=fieldName,proto3" json:"field_name,omitempty"` + FieldId int64 `protobuf:"varint,2,opt,name=field_id,json=fieldId,proto3" json:"field_id,omitempty"` + Texts []string `protobuf:"bytes,3,rep,name=texts,proto3" json:"texts,omitempty"` + AnalyzerNames []string `protobuf:"bytes,4,rep,name=analyzer_names,json=analyzerNames,proto3" json:"analyzer_names,omitempty"` // used if field with multi-analyzer + SearchTextNum int64 `protobuf:"varint,5,opt,name=search_text_num,json=searchTextNum,proto3" json:"search_text_num,omitempty"` + CorpusTextNum int64 `protobuf:"varint,6,opt,name=corpus_text_num,json=corpusTextNum,proto3" json:"corpus_text_num,omitempty"` +} + +func (x *HighlightTask) Reset() { + *x = HighlightTask{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[92] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *HighlightTask) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HighlightTask) ProtoMessage() {} + +func (x *HighlightTask) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[92] + 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 HighlightTask.ProtoReflect.Descriptor instead. +func (*HighlightTask) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{92} +} + +func (x *HighlightTask) GetFieldName() string { + if x != nil { + return x.FieldName + } + return "" +} + +func (x *HighlightTask) GetFieldId() int64 { + if x != nil { + return x.FieldId + } + return 0 +} + +func (x *HighlightTask) GetTexts() []string { + if x != nil { + return x.Texts + } + return nil +} + +func (x *HighlightTask) GetAnalyzerNames() []string { + if x != nil { + return x.AnalyzerNames + } + return nil +} + +func (x *HighlightTask) GetSearchTextNum() int64 { + if x != nil { + return x.SearchTextNum + } + return 0 +} + +func (x *HighlightTask) GetCorpusTextNum() int64 { + if x != nil { + return x.CorpusTextNum + } + return 0 +} + +type GetHighlightRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + Channel string `protobuf:"bytes,2,opt,name=channel,proto3" json:"channel,omitempty"` + Topks []int64 `protobuf:"varint,3,rep,packed,name=topks,proto3" json:"topks,omitempty"` + Tasks []*HighlightTask `protobuf:"bytes,4,rep,name=tasks,proto3" json:"tasks,omitempty"` // one task for one field +} + +func (x *GetHighlightRequest) Reset() { + *x = GetHighlightRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[93] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetHighlightRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetHighlightRequest) ProtoMessage() {} + +func (x *GetHighlightRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[93] + 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 GetHighlightRequest.ProtoReflect.Descriptor instead. +func (*GetHighlightRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{93} +} + +func (x *GetHighlightRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *GetHighlightRequest) GetChannel() string { + if x != nil { + return x.Channel + } + return "" +} + +func (x *GetHighlightRequest) GetTopks() []int64 { + if x != nil { + return x.Topks + } + return nil +} + +func (x *GetHighlightRequest) GetTasks() []*HighlightTask { + if x != nil { + return x.Tasks + } + return nil +} + +// start_offset and end_offset are fragment offset in the original text +// number of offsets always be 2 * number of highlight terms in the fragment +type HighlightFragment struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + StartOffset int64 `protobuf:"varint,1,opt,name=start_offset,json=startOffset,proto3" json:"start_offset,omitempty"` + EndOffset int64 `protobuf:"varint,2,opt,name=end_offset,json=endOffset,proto3" json:"end_offset,omitempty"` + Offsets []int64 `protobuf:"varint,3,rep,packed,name=offsets,proto3" json:"offsets,omitempty"` +} + +func (x *HighlightFragment) Reset() { + *x = HighlightFragment{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[94] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *HighlightFragment) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HighlightFragment) ProtoMessage() {} + +func (x *HighlightFragment) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[94] + 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 HighlightFragment.ProtoReflect.Descriptor instead. +func (*HighlightFragment) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{94} +} + +func (x *HighlightFragment) GetStartOffset() int64 { + if x != nil { + return x.StartOffset + } + return 0 +} + +func (x *HighlightFragment) GetEndOffset() int64 { + if x != nil { + return x.EndOffset + } + return 0 +} + +func (x *HighlightFragment) GetOffsets() []int64 { + if x != nil { + return x.Offsets + } + return nil +} + +type HighlightResult struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Fragments []*HighlightFragment `protobuf:"bytes,2,rep,name=fragments,proto3" json:"fragments,omitempty"` +} + +func (x *HighlightResult) Reset() { + *x = HighlightResult{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[95] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *HighlightResult) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*HighlightResult) ProtoMessage() {} + +func (x *HighlightResult) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[95] + 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 HighlightResult.ProtoReflect.Descriptor instead. +func (*HighlightResult) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{95} +} + +func (x *HighlightResult) GetFragments() []*HighlightFragment { + if x != nil { + return x.Fragments + } + return nil +} + +type GetHighlightResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + Results []*HighlightResult `protobuf:"bytes,2,rep,name=results,proto3" json:"results,omitempty"` +} + +func (x *GetHighlightResponse) Reset() { + *x = GetHighlightResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[96] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *GetHighlightResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*GetHighlightResponse) ProtoMessage() {} + +func (x *GetHighlightResponse) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[96] + 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 GetHighlightResponse.ProtoReflect.Descriptor instead. +func (*GetHighlightResponse) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{96} +} + +func (x *GetHighlightResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *GetHighlightResponse) GetResults() []*HighlightResult { + if x != nil { + return x.Results + } + return nil +} + type ListLoadedSegmentsRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -7774,7 +8101,7 @@ type ListLoadedSegmentsRequest struct { func (x *ListLoadedSegmentsRequest) Reset() { *x = ListLoadedSegmentsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[92] + mi := &file_query_coord_proto_msgTypes[97] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7787,7 +8114,7 @@ func (x *ListLoadedSegmentsRequest) String() string { func (*ListLoadedSegmentsRequest) ProtoMessage() {} func (x *ListLoadedSegmentsRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[92] + mi := &file_query_coord_proto_msgTypes[97] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7800,7 +8127,7 @@ func (x *ListLoadedSegmentsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ListLoadedSegmentsRequest.ProtoReflect.Descriptor instead. func (*ListLoadedSegmentsRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{92} + return file_query_coord_proto_rawDescGZIP(), []int{97} } func (x *ListLoadedSegmentsRequest) GetBase() *commonpb.MsgBase { @@ -7822,7 +8149,7 @@ type ListLoadedSegmentsResponse struct { func (x *ListLoadedSegmentsResponse) Reset() { *x = ListLoadedSegmentsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[93] + mi := &file_query_coord_proto_msgTypes[98] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7835,7 +8162,7 @@ func (x *ListLoadedSegmentsResponse) String() string { func (*ListLoadedSegmentsResponse) ProtoMessage() {} func (x *ListLoadedSegmentsResponse) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[93] + mi := &file_query_coord_proto_msgTypes[98] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7848,7 +8175,7 @@ func (x *ListLoadedSegmentsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ListLoadedSegmentsResponse.ProtoReflect.Descriptor instead. func (*ListLoadedSegmentsResponse) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{93} + return file_query_coord_proto_rawDescGZIP(), []int{98} } func (x *ListLoadedSegmentsResponse) GetStatus() *commonpb.Status { @@ -7880,7 +8207,7 @@ type DropIndexRequest struct { func (x *DropIndexRequest) Reset() { *x = DropIndexRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[94] + mi := &file_query_coord_proto_msgTypes[99] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7893,7 +8220,7 @@ func (x *DropIndexRequest) String() string { func (*DropIndexRequest) ProtoMessage() {} func (x *DropIndexRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[94] + mi := &file_query_coord_proto_msgTypes[99] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7906,7 +8233,7 @@ func (x *DropIndexRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DropIndexRequest.ProtoReflect.Descriptor instead. func (*DropIndexRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{94} + return file_query_coord_proto_rawDescGZIP(), []int{99} } func (x *DropIndexRequest) GetBase() *commonpb.MsgBase { @@ -9514,500 +9841,552 @@ var file_query_coord_proto_rawDesc = []byte{ 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0d, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x49, 0x6e, - 0x66, 0x6f, 0x73, 0x22, 0x4d, 0x0a, 0x19, 0x4c, 0x69, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x65, - 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 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, 0x22, 0x71, 0x0a, 0x1a, 0x4c, 0x69, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x64, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x49, 0x44, 0x73, 0x22, 0xbd, 0x01, 0x0a, 0x10, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, - 0x64, 0x65, 0x78, 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, 0x1c, 0x0a, 0x09, - 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x1a, 0x0a, 0x08, 0x69, 0x6e, - 0x64, 0x65, 0x78, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x08, 0x69, 0x6e, - 0x64, 0x65, 0x78, 0x49, 0x44, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, - 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, - 0x12, 0x23, 0x0a, 0x0d, 0x6e, 0x65, 0x65, 0x64, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, - 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x6e, 0x65, 0x65, 0x64, 0x54, 0x72, 0x61, - 0x6e, 0x73, 0x66, 0x65, 0x72, 0x2a, 0x36, 0x0a, 0x09, 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x6f, - 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x46, 0x75, 0x6c, 0x6c, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, - 0x44, 0x65, 0x6c, 0x74, 0x61, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, - 0x10, 0x02, 0x12, 0x09, 0x0a, 0x05, 0x53, 0x74, 0x61, 0x74, 0x73, 0x10, 0x03, 0x2a, 0x40, 0x0a, - 0x09, 0x44, 0x61, 0x74, 0x61, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, - 0x4b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x6c, 0x6c, 0x10, 0x01, - 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, - 0x0e, 0x0a, 0x0a, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x69, 0x63, 0x61, 0x6c, 0x10, 0x03, 0x2a, - 0x7a, 0x0a, 0x0e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x4e, 0x6f, 0x74, 0x45, 0x78, 0x69, 0x73, 0x74, 0x10, 0x00, 0x12, - 0x0e, 0x0a, 0x0a, 0x4e, 0x6f, 0x74, 0x50, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x10, 0x01, 0x12, - 0x0a, 0x0a, 0x06, 0x4f, 0x6e, 0x44, 0x69, 0x73, 0x6b, 0x10, 0x02, 0x12, 0x13, 0x0a, 0x0f, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x10, 0x03, - 0x12, 0x0c, 0x0a, 0x08, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x10, 0x04, 0x12, 0x10, - 0x0a, 0x0c, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x47, 0x50, 0x55, 0x10, 0x05, - 0x12, 0x09, 0x0a, 0x05, 0x49, 0x6e, 0x47, 0x50, 0x55, 0x10, 0x06, 0x2a, 0x64, 0x0a, 0x10, 0x54, - 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x13, 0x0a, 0x0f, 0x55, 0x6e, 0x4b, 0x6e, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x48, 0x61, 0x6e, 0x64, 0x6f, 0x66, 0x66, 0x10, - 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, - 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x47, 0x72, 0x70, 0x63, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x6f, 0x77, 0x6e, 0x10, - 0x04, 0x2a, 0x42, 0x0a, 0x08, 0x4c, 0x6f, 0x61, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, - 0x0b, 0x55, 0x6e, 0x4b, 0x6e, 0x6f, 0x77, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x10, 0x00, 0x12, 0x11, - 0x0a, 0x0d, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, - 0x01, 0x12, 0x12, 0x0a, 0x0e, 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x10, 0x02, 0x2a, 0x32, 0x0a, 0x0a, 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x74, 0x61, - 0x74, 0x75, 0x73, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, - 0x12, 0x0b, 0x0a, 0x07, 0x4c, 0x6f, 0x61, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x0a, 0x0a, - 0x06, 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x10, 0x02, 0x2a, 0x57, 0x0a, 0x08, 0x53, 0x79, 0x6e, - 0x63, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x10, - 0x00, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x65, 0x74, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x41, 0x6d, - 0x65, 0x6e, 0x64, 0x10, 0x02, 0x12, 0x11, 0x0a, 0x0d, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, - 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x12, 0x18, 0x0a, 0x14, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, - 0x10, 0x04, 0x32, 0xc4, 0x1f, 0x0a, 0x0a, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6f, 0x72, - 0x64, 0x12, 0x70, 0x0a, 0x13, 0x53, 0x68, 0x6f, 0x77, 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x68, - 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, - 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x12, 0x53, 0x68, 0x6f, 0x77, 0x4c, 0x6f, 0x61, 0x64, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, - 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0e, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, - 0x72, 0x74, 0x69, 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, 0x60, - 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, + 0x66, 0x6f, 0x73, 0x22, 0xd6, 0x01, 0x0a, 0x0d, 0x48, 0x69, 0x67, 0x68, 0x6c, 0x69, 0x67, 0x68, + 0x74, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x12, + 0x14, 0x0a, 0x05, 0x74, 0x65, 0x78, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, + 0x74, 0x65, 0x78, 0x74, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, + 0x72, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x61, + 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x26, 0x0a, 0x0f, + 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x5f, 0x74, 0x65, 0x78, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x73, 0x65, 0x61, 0x72, 0x63, 0x68, 0x54, 0x65, 0x78, + 0x74, 0x4e, 0x75, 0x6d, 0x12, 0x26, 0x0a, 0x0f, 0x63, 0x6f, 0x72, 0x70, 0x75, 0x73, 0x5f, 0x74, + 0x65, 0x78, 0x74, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, + 0x6f, 0x72, 0x70, 0x75, 0x73, 0x54, 0x65, 0x78, 0x74, 0x4e, 0x75, 0x6d, 0x22, 0xb0, 0x01, 0x0a, + 0x13, 0x47, 0x65, 0x74, 0x48, 0x69, 0x67, 0x68, 0x6c, 0x69, 0x67, 0x68, 0x74, 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, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x6b, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, + 0x05, 0x74, 0x6f, 0x70, 0x6b, 0x73, 0x12, 0x37, 0x0a, 0x05, 0x74, 0x61, 0x73, 0x6b, 0x73, 0x18, + 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x48, 0x69, 0x67, 0x68, 0x6c, + 0x69, 0x67, 0x68, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x52, 0x05, 0x74, 0x61, 0x73, 0x6b, 0x73, 0x22, + 0x6f, 0x0a, 0x11, 0x48, 0x69, 0x67, 0x68, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x46, 0x72, 0x61, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6f, 0x66, + 0x66, 0x73, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x72, + 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x65, 0x6e, 0x64, 0x5f, 0x6f, + 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x65, 0x6e, 0x64, + 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, + 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x73, + 0x22, 0x56, 0x0a, 0x0f, 0x48, 0x69, 0x67, 0x68, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x52, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x12, 0x43, 0x0a, 0x09, 0x66, 0x72, 0x61, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x48, 0x69, 0x67, 0x68, + 0x6c, 0x69, 0x67, 0x68, 0x74, 0x46, 0x72, 0x61, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x09, 0x66, + 0x72, 0x61, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x8a, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, + 0x48, 0x69, 0x67, 0x68, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3d, 0x0a, 0x07, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x48, 0x69, 0x67, + 0x68, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x52, 0x07, 0x72, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x4d, 0x0a, 0x19, 0x4c, 0x69, 0x73, 0x74, 0x4c, 0x6f, 0x61, + 0x64, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 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, 0x22, 0x71, 0x0a, 0x1a, 0x4c, 0x69, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, + 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x22, 0xbd, 0x01, 0x0a, 0x10, 0x44, 0x72, 0x6f, 0x70, + 0x49, 0x6e, 0x64, 0x65, 0x78, 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, 0x1c, + 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x1a, 0x0a, 0x08, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x08, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x12, 0x23, 0x0a, 0x0d, 0x6e, 0x65, 0x65, 0x64, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, + 0x66, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x6e, 0x65, 0x65, 0x64, 0x54, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x2a, 0x36, 0x0a, 0x09, 0x4c, 0x6f, 0x61, 0x64, 0x53, + 0x63, 0x6f, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x46, 0x75, 0x6c, 0x6c, 0x10, 0x00, 0x12, 0x09, + 0x0a, 0x05, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x10, 0x02, 0x12, 0x09, 0x0a, 0x05, 0x53, 0x74, 0x61, 0x74, 0x73, 0x10, 0x03, 0x2a, + 0x40, 0x0a, 0x09, 0x44, 0x61, 0x74, 0x61, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, + 0x55, 0x6e, 0x4b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x6c, 0x6c, + 0x10, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x10, + 0x02, 0x12, 0x0e, 0x0a, 0x0a, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x69, 0x63, 0x61, 0x6c, 0x10, + 0x03, 0x2a, 0x7a, 0x0a, 0x0e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x4e, 0x6f, 0x74, 0x45, 0x78, 0x69, 0x73, 0x74, 0x10, + 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x4e, 0x6f, 0x74, 0x50, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x10, + 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x4f, 0x6e, 0x44, 0x69, 0x73, 0x6b, 0x10, 0x02, 0x12, 0x13, 0x0a, + 0x0f, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, + 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x10, 0x04, + 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x47, 0x50, 0x55, + 0x10, 0x05, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x6e, 0x47, 0x50, 0x55, 0x10, 0x06, 0x2a, 0x64, 0x0a, + 0x10, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x13, 0x0a, 0x0f, 0x55, 0x6e, 0x4b, 0x6e, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x64, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x48, 0x61, 0x6e, 0x64, 0x6f, 0x66, + 0x66, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, + 0x63, 0x65, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x47, 0x72, 0x70, 0x63, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x6f, 0x77, + 0x6e, 0x10, 0x04, 0x2a, 0x42, 0x0a, 0x08, 0x4c, 0x6f, 0x61, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, + 0x0f, 0x0a, 0x0b, 0x55, 0x6e, 0x4b, 0x6e, 0x6f, 0x77, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x10, 0x00, + 0x12, 0x11, 0x0a, 0x0d, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x10, 0x01, 0x12, 0x12, 0x0a, 0x0e, 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, 0x2a, 0x32, 0x0a, 0x0a, 0x4c, 0x6f, 0x61, 0x64, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, + 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x4c, 0x6f, 0x61, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, + 0x0a, 0x0a, 0x06, 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x10, 0x02, 0x2a, 0x57, 0x0a, 0x08, 0x53, + 0x79, 0x6e, 0x63, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x52, 0x65, 0x6d, 0x6f, 0x76, + 0x65, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x65, 0x74, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, + 0x41, 0x6d, 0x65, 0x6e, 0x64, 0x10, 0x02, 0x12, 0x11, 0x0a, 0x0d, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x12, 0x18, 0x0a, 0x14, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x10, 0x04, 0x32, 0xc4, 0x1f, 0x0a, 0x0a, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, + 0x6f, 0x72, 0x64, 0x12, 0x70, 0x0a, 0x13, 0x53, 0x68, 0x6f, 0x77, 0x4c, 0x6f, 0x61, 0x64, 0x43, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x68, 0x6f, 0x77, + 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x12, 0x53, 0x68, 0x6f, 0x77, 0x4c, 0x6f, 0x61, + 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x68, 0x6f, 0x77, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0e, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 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, 0x5a, 0x0a, 0x0e, 0x4c, 0x6f, 0x61, 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, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 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, 0x60, 0x0a, 0x11, - 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 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, 0x6c, - 0x0a, 0x17, 0x53, 0x79, 0x6e, 0x63, 0x4e, 0x65, 0x77, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, - 0x79, 0x6e, 0x63, 0x4e, 0x65, 0x77, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x50, 0x61, 0x72, - 0x74, 0x69, 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, 0x75, 0x0a, 0x12, - 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 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, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, - 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 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, 0x71, 0x75, 0x65, 0x72, 0x79, 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, 0x54, 0x0a, 0x0b, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, - 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, - 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, 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, 0x62, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x52, 0x65, 0x70, - 0x6c, 0x69, 0x63, 0x61, 0x73, 0x12, 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, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 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, 0x47, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6c, 0x0a, 0x0f, 0x47, 0x65, - 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x2a, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, - 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, - 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x62, 0x0a, 0x0b, 0x43, 0x68, 0x65, 0x63, - 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, - 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, - 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x65, 0x0a, 0x13, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, - 0x6f, 0x75, 0x70, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 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, 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x12, 0x2f, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, - 0x72, 0x6f, 0x75, 0x70, 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, 0x61, 0x0a, 0x11, 0x44, - 0x72, 0x6f, 0x70, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, - 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x52, 0x65, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 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, 0x5a, - 0x0a, 0x0c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x28, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4e, 0x6f, 0x64, - 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, 0x03, 0x88, 0x02, 0x01, 0x12, 0x5c, 0x0a, 0x0f, 0x54, 0x72, - 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x12, 0x2a, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x52, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 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, 0x77, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, - 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x12, 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, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 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, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x7e, 0x0a, 0x15, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, - 0x79, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x75, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, 0x73, - 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74, - 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x0c, 0x4c, 0x69, 0x73, 0x74, - 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, - 0x73, 0x74, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x68, 0x65, 0x63, 0x6b, - 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5c, 0x0a, - 0x0f, 0x41, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, - 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, - 0x65, 0x63, 0x6b, 0x65, 0x72, 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, 0x60, 0x0a, 0x11, 0x44, - 0x65, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, - 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, - 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 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, 0x4c, 0x69, 0x73, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x28, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, - 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, - 0x73, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x87, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x51, 0x75, 0x65, - 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, - 0x6f, 0x6e, 0x12, 0x33, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, - 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x34, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, - 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, - 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x5a, 0x0a, 0x0e, 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, - 0x65, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x42, 0x61, - 0x6c, 0x61, 0x6e, 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, 0x58, 0x0a, 0x0d, 0x52, - 0x65, 0x73, 0x75, 0x6d, 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x28, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, - 0x79, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e, 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, 0x75, 0x0a, 0x12, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x42, 0x61, - 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x74, 0x61, - 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x43, 0x68, 0x65, 0x63, 0x6b, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0b, - 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x2e, 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, + 0x73, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 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, 0x52, 0x0a, 0x0a, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x4e, 0x6f, 0x64, 0x65, - 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x4e, 0x6f, 0x64, 0x65, + 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0e, 0x4c, 0x6f, 0x61, 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, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 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, 0x60, + 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, + 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, 0x6c, 0x0a, 0x17, 0x53, 0x79, 0x6e, 0x63, 0x4e, 0x65, 0x77, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x32, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x4e, 0x65, 0x77, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x50, + 0x61, 0x72, 0x74, 0x69, 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, 0x75, + 0x0a, 0x12, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 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, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x4c, 0x6f, 0x61, 0x64, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 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, 0x71, 0x75, 0x65, 0x72, 0x79, 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, 0x54, 0x0a, 0x0b, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, + 0x6e, 0x63, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, + 0x61, 0x6e, 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, 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, 0x62, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x52, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x12, 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, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 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, 0x47, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, + 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6c, 0x0a, 0x0f, + 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, + 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x62, 0x0a, 0x0b, 0x43, 0x68, + 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, + 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x65, + 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 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, 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x12, 0x2f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 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, 0x61, 0x0a, + 0x11, 0x44, 0x72, 0x6f, 0x70, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, + 0x75, 0x70, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x52, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 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, 0x5a, 0x0a, 0x0c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, + 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4e, + 0x6f, 0x64, 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, 0x03, 0x88, 0x02, 0x01, 0x12, 0x5c, 0x0a, 0x0f, + 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x12, + 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 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, 0x77, 0x0a, 0x12, 0x4c, 0x69, + 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, + 0x12, 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, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 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, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, 0x15, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x30, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x75, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x65, + 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, + 0x69, 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, + 0x73, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x0c, 0x4c, 0x69, + 0x73, 0x74, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x68, 0x65, + 0x63, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x5c, 0x0a, 0x0f, 0x41, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, + 0x65, 0x72, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, + 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 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, 0x60, 0x0a, + 0x11, 0x44, 0x65, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, + 0x65, 0x72, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, + 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 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, 0x4c, 0x69, 0x73, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, + 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, + 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x4c, 0x69, 0x73, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x87, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x51, + 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x33, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x51, 0x75, 0x65, + 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x34, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, + 0x65, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x5a, 0x0a, 0x0e, 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x42, 0x61, 0x6c, 0x61, + 0x6e, 0x63, 0x65, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, + 0x42, 0x61, 0x6c, 0x61, 0x6e, 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, 0x58, 0x0a, + 0x0d, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x28, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e, 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, 0x75, 0x0a, 0x12, 0x43, 0x68, 0x65, 0x63, 0x6b, + 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2d, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x54, + 0x0a, 0x0b, 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x26, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x4e, 0x6f, 0x64, 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, 0x52, 0x0a, 0x0a, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x4e, 0x6f, + 0x64, 0x65, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x4e, 0x6f, + 0x64, 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, 0x5c, 0x0a, 0x0f, 0x54, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x2a, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 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, 0x5c, 0x0a, 0x0f, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, - 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, - 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 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, 0x5c, 0x0a, 0x0f, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, - 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x72, 0x61, - 0x6e, 0x73, 0x66, 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 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, 0x72, 0x0a, 0x1a, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x51, 0x75, 0x65, 0x72, 0x79, - 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x35, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x51, 0x75, 0x65, 0x72, 0x79, - 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 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, 0x5e, 0x0a, 0x10, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, - 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 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, 0x0b, 0x52, 0x75, 0x6e, 0x41, 0x6e, 0x61, - 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x75, 0x6e, 0x41, 0x6e, - 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 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, 0x75, 0x6e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x10, 0x56, 0x61, 0x6c, - 0x69, 0x64, 0x61, 0x74, 0x65, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x12, 0x2b, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x41, 0x6e, 0x61, 0x6c, 0x79, - 0x7a, 0x65, 0x72, 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, 0x32, 0xa0, 0x16, 0x0a, 0x09, 0x51, 0x75, - 0x65, 0x72, 0x79, 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, 0x6d, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x54, 0x69, 0x6d, 0x65, - 0x54, 0x69, 0x63, 0x6b, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 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, 0x47, 0x65, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 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, 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, 0x5c, 0x0a, 0x0f, 0x57, 0x61, 0x74, 0x63, 0x68, - 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 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, 0x5a, 0x0a, 0x0e, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x44, 0x6d, - 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x55, 0x6e, 0x73, - 0x75, 0x62, 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 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, 0x56, 0x0a, 0x0c, 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 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, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, - 0x65, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, - 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, + 0x75, 0x73, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x1a, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x51, 0x75, 0x65, + 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x35, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x51, 0x75, 0x65, + 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 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, 0x5e, 0x0a, 0x10, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 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, 0x0b, 0x52, 0x75, 0x6e, 0x41, + 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x75, 0x6e, + 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 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, 0x75, 0x6e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, + 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x10, 0x56, + 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x12, + 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x41, 0x6e, 0x61, + 0x6c, 0x79, 0x7a, 0x65, 0x72, 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, 0x5a, 0x0a, 0x0e, 0x4c, - 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x29, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x32, 0x85, 0x17, 0x0a, 0x09, + 0x51, 0x75, 0x65, 0x72, 0x79, 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, 0x6d, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x54, 0x69, + 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 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, 0x47, 0x65, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, + 0x6b, 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, 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, 0x5c, 0x0a, 0x0f, 0x57, 0x61, 0x74, + 0x63, 0x68, 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x2a, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, + 0x79, 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, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x65, 0x61, - 0x73, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2c, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, - 0x79, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0e, 0x55, 0x6e, 0x73, 0x75, 0x62, + 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x55, + 0x6e, 0x73, 0x75, 0x62, 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 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, 0x56, 0x0a, 0x0c, 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 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, 0x60, 0x0a, 0x11, 0x52, + 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 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, 0x5a, 0x0a, + 0x0e, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, + 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 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, 0x5c, 0x0a, 0x0f, 0x52, 0x65, 0x6c, - 0x65, 0x61, 0x73, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2a, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, - 0x79, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 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, 0x69, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x53, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, - 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 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, 0x71, 0x75, 0x65, 0x72, 0x79, 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, 0x64, 0x0a, 0x13, 0x53, 0x79, 0x6e, 0x63, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, - 0x79, 0x6e, 0x63, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 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, 0x69, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x53, - 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, - 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x2c, 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, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x53, 0x0a, 0x06, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x12, 0x21, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, - 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x0e, 0x53, 0x65, 0x61, 0x72, - 0x63, 0x68, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x21, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, - 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x73, 0x22, 0x00, 0x12, 0x53, 0x0a, 0x05, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x20, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, + 0x65, 0x61, 0x73, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, - 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, - 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x0b, 0x51, 0x75, - 0x65, 0x72, 0x79, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, - 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, - 0x6e, 0x61, 0x6c, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x52, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x73, 0x22, 0x00, 0x30, 0x01, 0x12, 0x5b, 0x0a, 0x0d, 0x51, 0x75, 0x65, 0x72, 0x79, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, - 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, - 0x61, 0x6c, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, - 0x74, 0x73, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x13, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x74, 0x72, - 0x65, 0x61, 0x6d, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x20, 0x2e, 0x6d, 0x69, + 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, + 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, 0x5c, 0x0a, 0x0f, 0x52, + 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2a, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 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, 0x69, 0x0a, 0x0e, 0x47, 0x65, 0x74, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 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, 0x71, 0x75, 0x65, 0x72, 0x79, 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, 0x64, 0x0a, 0x13, 0x53, 0x79, 0x6e, 0x63, 0x52, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 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, 0x69, 0x0a, 0x0d, 0x47, 0x65, + 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x28, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 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, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x53, 0x0a, 0x06, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x12, + 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, + 0x68, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x0e, 0x53, 0x65, + 0x61, 0x72, 0x63, 0x68, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x21, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x00, 0x12, 0x53, 0x0a, 0x05, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x69, + 0x65, 0x76, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x0b, + 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x52, 0x65, - 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x00, 0x30, 0x01, 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, + 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x00, 0x30, 0x01, 0x12, 0x5b, 0x0a, 0x0d, 0x51, 0x75, 0x65, + 0x72, 0x79, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x52, 0x65, 0x73, + 0x75, 0x6c, 0x74, 0x73, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x13, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x20, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, + 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x00, 0x30, 0x01, 0x12, 0x7b, 0x0a, 0x12, 0x53, + 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, - 0x72, 0x69, 0x63, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, - 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x78, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x44, 0x61, - 0x74, 0x61, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2e, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, - 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x44, 0x69, 0x73, 0x74, 0x72, - 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, - 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x44, 0x69, 0x73, 0x74, 0x72, - 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x5e, 0x0a, 0x10, 0x53, 0x79, 0x6e, 0x63, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, - 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x44, - 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 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, 0x4a, 0x0a, 0x06, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x21, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 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, 0x60, 0x0a, - 0x0b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x26, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, - 0x79, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, - 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x56, 0x0a, 0x0c, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, - 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, - 0x61, 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, 0x0b, 0x52, 0x75, 0x6e, 0x41, 0x6e, - 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x75, 0x6e, 0x41, - 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, + 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, 0x52, 0x75, 0x6e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, - 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, 0x71, 0x75, 0x65, 0x72, 0x79, 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, 0x5e, 0x0a, 0x10, - 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, - 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x41, 0x6e, - 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 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, 0x32, 0x5a, 0x30, - 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, - 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x70, 0x62, - 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 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, 0x78, 0x0a, 0x13, 0x47, 0x65, 0x74, + 0x44, 0x61, 0x74, 0x61, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x44, 0x69, 0x73, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x44, 0x69, 0x73, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x10, 0x53, 0x79, 0x6e, 0x63, 0x44, 0x69, 0x73, 0x74, 0x72, + 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x79, 0x6e, + 0x63, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 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, 0x4a, 0x0a, 0x06, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x21, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 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, + 0x60, 0x0a, 0x0b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x26, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x56, 0x0a, 0x0c, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, + 0x61, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, + 0x65, 0x6d, 0x61, 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, 0x0b, 0x52, 0x75, 0x6e, + 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x75, + 0x6e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 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, 0x75, 0x6e, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, + 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x0c, + 0x47, 0x65, 0x74, 0x48, 0x69, 0x67, 0x68, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x12, 0x27, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x2e, 0x47, 0x65, 0x74, 0x48, 0x69, 0x67, 0x68, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x48, 0x69, + 0x67, 0x68, 0x6c, 0x69, 0x67, 0x68, 0x74, 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, 0x71, 0x75, + 0x65, 0x72, 0x79, 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, 0x5e, 0x0a, 0x10, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x41, + 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x56, 0x61, 0x6c, + 0x69, 0x64, 0x61, 0x74, 0x65, 0x41, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 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, 0x32, 0x5a, 0x30, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, + 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -10023,7 +10402,7 @@ func file_query_coord_proto_rawDescGZIP() []byte { } var file_query_coord_proto_enumTypes = make([]protoimpl.EnumInfo, 7) -var file_query_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 116) +var file_query_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 121) var file_query_coord_proto_goTypes = []interface{}{ (LoadScope)(0), // 0: milvus.proto.query.LoadScope (DataScope)(0), // 1: milvus.proto.query.DataScope @@ -10124,398 +10503,410 @@ var file_query_coord_proto_goTypes = []interface{}{ (*RunAnalyzerRequest)(nil), // 96: milvus.proto.query.RunAnalyzerRequest (*AnalyzerInfo)(nil), // 97: milvus.proto.query.AnalyzerInfo (*ValidateAnalyzerRequest)(nil), // 98: milvus.proto.query.ValidateAnalyzerRequest - (*ListLoadedSegmentsRequest)(nil), // 99: milvus.proto.query.ListLoadedSegmentsRequest - (*ListLoadedSegmentsResponse)(nil), // 100: milvus.proto.query.ListLoadedSegmentsResponse - (*DropIndexRequest)(nil), // 101: milvus.proto.query.DropIndexRequest - nil, // 102: milvus.proto.query.LoadCollectionRequest.FieldIndexIDEntry - nil, // 103: milvus.proto.query.LoadPartitionsRequest.FieldIndexIDEntry - nil, // 104: milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry - nil, // 105: milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry - nil, // 106: milvus.proto.query.WatchDmChannelsRequest.SealedSegmentRowCountEntry - nil, // 107: milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry - nil, // 108: milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry - nil, // 109: milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry - nil, // 110: milvus.proto.query.LeaderView.SegmentDistEntry - nil, // 111: milvus.proto.query.LeaderView.GrowingSegmentsEntry - nil, // 112: milvus.proto.query.LeaderView.PartitionStatsVersionsEntry - nil, // 113: milvus.proto.query.SegmentVersionInfo.IndexInfoEntry - nil, // 114: milvus.proto.query.SegmentVersionInfo.JsonStatsInfoEntry - nil, // 115: milvus.proto.query.CollectionLoadInfo.FieldIndexIDEntry - nil, // 116: milvus.proto.query.PartitionLoadInfo.FieldIndexIDEntry - nil, // 117: milvus.proto.query.Replica.ChannelNodeInfosEntry - nil, // 118: milvus.proto.query.SyncAction.PartitionStatsVersionsEntry - nil, // 119: milvus.proto.query.SyncAction.SealedSegmentRowCountEntry - nil, // 120: milvus.proto.query.ResourceGroupInfo.NumLoadedReplicaEntry - nil, // 121: milvus.proto.query.ResourceGroupInfo.NumOutgoingNodeEntry - nil, // 122: milvus.proto.query.ResourceGroupInfo.NumIncomingNodeEntry - (*commonpb.MsgBase)(nil), // 123: milvus.proto.common.MsgBase - (*commonpb.Status)(nil), // 124: milvus.proto.common.Status - (*schemapb.LongArray)(nil), // 125: milvus.proto.schema.LongArray - (*schemapb.CollectionSchema)(nil), // 126: milvus.proto.schema.CollectionSchema - (commonpb.LoadPriority)(0), // 127: milvus.proto.common.LoadPriority - (*internalpb.GetStatisticsRequest)(nil), // 128: milvus.proto.internal.GetStatisticsRequest - (*indexpb.IndexInfo)(nil), // 129: milvus.proto.index.IndexInfo - (*commonpb.KeyValuePair)(nil), // 130: milvus.proto.common.KeyValuePair - (*datapb.VchannelInfo)(nil), // 131: milvus.proto.data.VchannelInfo - (*datapb.SegmentInfo)(nil), // 132: milvus.proto.data.SegmentInfo - (*datapb.FieldBinlog)(nil), // 133: milvus.proto.data.FieldBinlog - (*msgpb.MsgPosition)(nil), // 134: milvus.proto.msg.MsgPosition - (datapb.SegmentLevel)(0), // 135: milvus.proto.data.SegmentLevel - (*internalpb.SearchRequest)(nil), // 136: milvus.proto.internal.SearchRequest - (*internalpb.RetrieveRequest)(nil), // 137: milvus.proto.internal.RetrieveRequest - (commonpb.SegmentState)(0), // 138: milvus.proto.common.SegmentState - (*rgpb.ResourceGroupConfig)(nil), // 139: milvus.proto.rg.ResourceGroupConfig - (*commonpb.NodeInfo)(nil), // 140: milvus.proto.common.NodeInfo - (*schemapb.IDs)(nil), // 141: milvus.proto.schema.IDs - (*datapb.TextIndexStats)(nil), // 142: milvus.proto.data.TextIndexStats - (*datapb.JsonKeyStats)(nil), // 143: milvus.proto.data.JsonKeyStats - (*internalpb.ShowConfigurationsRequest)(nil), // 144: milvus.proto.internal.ShowConfigurationsRequest - (*milvuspb.GetMetricsRequest)(nil), // 145: milvus.proto.milvus.GetMetricsRequest - (*milvuspb.GetReplicasRequest)(nil), // 146: milvus.proto.milvus.GetReplicasRequest - (*milvuspb.CheckHealthRequest)(nil), // 147: milvus.proto.milvus.CheckHealthRequest - (*milvuspb.CreateResourceGroupRequest)(nil), // 148: milvus.proto.milvus.CreateResourceGroupRequest - (*milvuspb.DropResourceGroupRequest)(nil), // 149: milvus.proto.milvus.DropResourceGroupRequest - (*milvuspb.TransferNodeRequest)(nil), // 150: milvus.proto.milvus.TransferNodeRequest - (*milvuspb.ListResourceGroupsRequest)(nil), // 151: milvus.proto.milvus.ListResourceGroupsRequest - (*milvuspb.GetComponentStatesRequest)(nil), // 152: milvus.proto.milvus.GetComponentStatesRequest - (*internalpb.GetTimeTickChannelRequest)(nil), // 153: milvus.proto.internal.GetTimeTickChannelRequest - (*internalpb.GetStatisticsChannelRequest)(nil), // 154: milvus.proto.internal.GetStatisticsChannelRequest - (*internalpb.ShowConfigurationsResponse)(nil), // 155: milvus.proto.internal.ShowConfigurationsResponse - (*milvuspb.GetMetricsResponse)(nil), // 156: milvus.proto.milvus.GetMetricsResponse - (*milvuspb.GetReplicasResponse)(nil), // 157: milvus.proto.milvus.GetReplicasResponse - (*milvuspb.CheckHealthResponse)(nil), // 158: milvus.proto.milvus.CheckHealthResponse - (*milvuspb.ListResourceGroupsResponse)(nil), // 159: milvus.proto.milvus.ListResourceGroupsResponse - (*milvuspb.RunAnalyzerResponse)(nil), // 160: milvus.proto.milvus.RunAnalyzerResponse - (*milvuspb.ComponentStates)(nil), // 161: milvus.proto.milvus.ComponentStates - (*milvuspb.StringResponse)(nil), // 162: milvus.proto.milvus.StringResponse - (*internalpb.GetStatisticsResponse)(nil), // 163: milvus.proto.internal.GetStatisticsResponse - (*internalpb.SearchResults)(nil), // 164: milvus.proto.internal.SearchResults - (*internalpb.RetrieveResults)(nil), // 165: milvus.proto.internal.RetrieveResults + (*HighlightTask)(nil), // 99: milvus.proto.query.HighlightTask + (*GetHighlightRequest)(nil), // 100: milvus.proto.query.GetHighlightRequest + (*HighlightFragment)(nil), // 101: milvus.proto.query.HighlightFragment + (*HighlightResult)(nil), // 102: milvus.proto.query.HighlightResult + (*GetHighlightResponse)(nil), // 103: milvus.proto.query.GetHighlightResponse + (*ListLoadedSegmentsRequest)(nil), // 104: milvus.proto.query.ListLoadedSegmentsRequest + (*ListLoadedSegmentsResponse)(nil), // 105: milvus.proto.query.ListLoadedSegmentsResponse + (*DropIndexRequest)(nil), // 106: milvus.proto.query.DropIndexRequest + nil, // 107: milvus.proto.query.LoadCollectionRequest.FieldIndexIDEntry + nil, // 108: milvus.proto.query.LoadPartitionsRequest.FieldIndexIDEntry + nil, // 109: milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry + nil, // 110: milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry + nil, // 111: milvus.proto.query.WatchDmChannelsRequest.SealedSegmentRowCountEntry + nil, // 112: milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry + nil, // 113: milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry + nil, // 114: milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry + nil, // 115: milvus.proto.query.LeaderView.SegmentDistEntry + nil, // 116: milvus.proto.query.LeaderView.GrowingSegmentsEntry + nil, // 117: milvus.proto.query.LeaderView.PartitionStatsVersionsEntry + nil, // 118: milvus.proto.query.SegmentVersionInfo.IndexInfoEntry + nil, // 119: milvus.proto.query.SegmentVersionInfo.JsonStatsInfoEntry + nil, // 120: milvus.proto.query.CollectionLoadInfo.FieldIndexIDEntry + nil, // 121: milvus.proto.query.PartitionLoadInfo.FieldIndexIDEntry + nil, // 122: milvus.proto.query.Replica.ChannelNodeInfosEntry + nil, // 123: milvus.proto.query.SyncAction.PartitionStatsVersionsEntry + nil, // 124: milvus.proto.query.SyncAction.SealedSegmentRowCountEntry + nil, // 125: milvus.proto.query.ResourceGroupInfo.NumLoadedReplicaEntry + nil, // 126: milvus.proto.query.ResourceGroupInfo.NumOutgoingNodeEntry + nil, // 127: milvus.proto.query.ResourceGroupInfo.NumIncomingNodeEntry + (*commonpb.MsgBase)(nil), // 128: milvus.proto.common.MsgBase + (*commonpb.Status)(nil), // 129: milvus.proto.common.Status + (*schemapb.LongArray)(nil), // 130: milvus.proto.schema.LongArray + (*schemapb.CollectionSchema)(nil), // 131: milvus.proto.schema.CollectionSchema + (commonpb.LoadPriority)(0), // 132: milvus.proto.common.LoadPriority + (*internalpb.GetStatisticsRequest)(nil), // 133: milvus.proto.internal.GetStatisticsRequest + (*indexpb.IndexInfo)(nil), // 134: milvus.proto.index.IndexInfo + (*commonpb.KeyValuePair)(nil), // 135: milvus.proto.common.KeyValuePair + (*datapb.VchannelInfo)(nil), // 136: milvus.proto.data.VchannelInfo + (*datapb.SegmentInfo)(nil), // 137: milvus.proto.data.SegmentInfo + (*datapb.FieldBinlog)(nil), // 138: milvus.proto.data.FieldBinlog + (*msgpb.MsgPosition)(nil), // 139: milvus.proto.msg.MsgPosition + (datapb.SegmentLevel)(0), // 140: milvus.proto.data.SegmentLevel + (*internalpb.SearchRequest)(nil), // 141: milvus.proto.internal.SearchRequest + (*internalpb.RetrieveRequest)(nil), // 142: milvus.proto.internal.RetrieveRequest + (commonpb.SegmentState)(0), // 143: milvus.proto.common.SegmentState + (*rgpb.ResourceGroupConfig)(nil), // 144: milvus.proto.rg.ResourceGroupConfig + (*commonpb.NodeInfo)(nil), // 145: milvus.proto.common.NodeInfo + (*schemapb.IDs)(nil), // 146: milvus.proto.schema.IDs + (*datapb.TextIndexStats)(nil), // 147: milvus.proto.data.TextIndexStats + (*datapb.JsonKeyStats)(nil), // 148: milvus.proto.data.JsonKeyStats + (*internalpb.ShowConfigurationsRequest)(nil), // 149: milvus.proto.internal.ShowConfigurationsRequest + (*milvuspb.GetMetricsRequest)(nil), // 150: milvus.proto.milvus.GetMetricsRequest + (*milvuspb.GetReplicasRequest)(nil), // 151: milvus.proto.milvus.GetReplicasRequest + (*milvuspb.CheckHealthRequest)(nil), // 152: milvus.proto.milvus.CheckHealthRequest + (*milvuspb.CreateResourceGroupRequest)(nil), // 153: milvus.proto.milvus.CreateResourceGroupRequest + (*milvuspb.DropResourceGroupRequest)(nil), // 154: milvus.proto.milvus.DropResourceGroupRequest + (*milvuspb.TransferNodeRequest)(nil), // 155: milvus.proto.milvus.TransferNodeRequest + (*milvuspb.ListResourceGroupsRequest)(nil), // 156: milvus.proto.milvus.ListResourceGroupsRequest + (*milvuspb.GetComponentStatesRequest)(nil), // 157: milvus.proto.milvus.GetComponentStatesRequest + (*internalpb.GetTimeTickChannelRequest)(nil), // 158: milvus.proto.internal.GetTimeTickChannelRequest + (*internalpb.GetStatisticsChannelRequest)(nil), // 159: milvus.proto.internal.GetStatisticsChannelRequest + (*internalpb.ShowConfigurationsResponse)(nil), // 160: milvus.proto.internal.ShowConfigurationsResponse + (*milvuspb.GetMetricsResponse)(nil), // 161: milvus.proto.milvus.GetMetricsResponse + (*milvuspb.GetReplicasResponse)(nil), // 162: milvus.proto.milvus.GetReplicasResponse + (*milvuspb.CheckHealthResponse)(nil), // 163: milvus.proto.milvus.CheckHealthResponse + (*milvuspb.ListResourceGroupsResponse)(nil), // 164: milvus.proto.milvus.ListResourceGroupsResponse + (*milvuspb.RunAnalyzerResponse)(nil), // 165: milvus.proto.milvus.RunAnalyzerResponse + (*milvuspb.ComponentStates)(nil), // 166: milvus.proto.milvus.ComponentStates + (*milvuspb.StringResponse)(nil), // 167: milvus.proto.milvus.StringResponse + (*internalpb.GetStatisticsResponse)(nil), // 168: milvus.proto.internal.GetStatisticsResponse + (*internalpb.SearchResults)(nil), // 169: milvus.proto.internal.SearchResults + (*internalpb.RetrieveResults)(nil), // 170: milvus.proto.internal.RetrieveResults } var file_query_coord_proto_depIdxs = []int32{ - 123, // 0: milvus.proto.query.ShowCollectionsRequest.base:type_name -> milvus.proto.common.MsgBase - 124, // 1: milvus.proto.query.ShowCollectionsResponse.status:type_name -> milvus.proto.common.Status - 125, // 2: milvus.proto.query.ShowCollectionsResponse.load_fields:type_name -> milvus.proto.schema.LongArray - 123, // 3: milvus.proto.query.ShowPartitionsRequest.base:type_name -> milvus.proto.common.MsgBase - 124, // 4: milvus.proto.query.ShowPartitionsResponse.status:type_name -> milvus.proto.common.Status - 123, // 5: milvus.proto.query.LoadCollectionRequest.base:type_name -> milvus.proto.common.MsgBase - 126, // 6: milvus.proto.query.LoadCollectionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 102, // 7: milvus.proto.query.LoadCollectionRequest.field_indexID:type_name -> milvus.proto.query.LoadCollectionRequest.FieldIndexIDEntry - 127, // 8: milvus.proto.query.LoadCollectionRequest.priority:type_name -> milvus.proto.common.LoadPriority - 123, // 9: milvus.proto.query.ReleaseCollectionRequest.base:type_name -> milvus.proto.common.MsgBase - 128, // 10: milvus.proto.query.GetStatisticsRequest.req:type_name -> milvus.proto.internal.GetStatisticsRequest + 128, // 0: milvus.proto.query.ShowCollectionsRequest.base:type_name -> milvus.proto.common.MsgBase + 129, // 1: milvus.proto.query.ShowCollectionsResponse.status:type_name -> milvus.proto.common.Status + 130, // 2: milvus.proto.query.ShowCollectionsResponse.load_fields:type_name -> milvus.proto.schema.LongArray + 128, // 3: milvus.proto.query.ShowPartitionsRequest.base:type_name -> milvus.proto.common.MsgBase + 129, // 4: milvus.proto.query.ShowPartitionsResponse.status:type_name -> milvus.proto.common.Status + 128, // 5: milvus.proto.query.LoadCollectionRequest.base:type_name -> milvus.proto.common.MsgBase + 131, // 6: milvus.proto.query.LoadCollectionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 107, // 7: milvus.proto.query.LoadCollectionRequest.field_indexID:type_name -> milvus.proto.query.LoadCollectionRequest.FieldIndexIDEntry + 132, // 8: milvus.proto.query.LoadCollectionRequest.priority:type_name -> milvus.proto.common.LoadPriority + 128, // 9: milvus.proto.query.ReleaseCollectionRequest.base:type_name -> milvus.proto.common.MsgBase + 133, // 10: milvus.proto.query.GetStatisticsRequest.req:type_name -> milvus.proto.internal.GetStatisticsRequest 1, // 11: milvus.proto.query.GetStatisticsRequest.scope:type_name -> milvus.proto.query.DataScope - 123, // 12: milvus.proto.query.LoadPartitionsRequest.base:type_name -> milvus.proto.common.MsgBase - 126, // 13: milvus.proto.query.LoadPartitionsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 103, // 14: milvus.proto.query.LoadPartitionsRequest.field_indexID:type_name -> milvus.proto.query.LoadPartitionsRequest.FieldIndexIDEntry - 129, // 15: milvus.proto.query.LoadPartitionsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo - 127, // 16: milvus.proto.query.LoadPartitionsRequest.priority:type_name -> milvus.proto.common.LoadPriority - 123, // 17: milvus.proto.query.ReleasePartitionsRequest.base:type_name -> milvus.proto.common.MsgBase - 123, // 18: milvus.proto.query.GetPartitionStatesRequest.base:type_name -> milvus.proto.common.MsgBase - 124, // 19: milvus.proto.query.GetPartitionStatesResponse.status:type_name -> milvus.proto.common.Status + 128, // 12: milvus.proto.query.LoadPartitionsRequest.base:type_name -> milvus.proto.common.MsgBase + 131, // 13: milvus.proto.query.LoadPartitionsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 108, // 14: milvus.proto.query.LoadPartitionsRequest.field_indexID:type_name -> milvus.proto.query.LoadPartitionsRequest.FieldIndexIDEntry + 134, // 15: milvus.proto.query.LoadPartitionsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 132, // 16: milvus.proto.query.LoadPartitionsRequest.priority:type_name -> milvus.proto.common.LoadPriority + 128, // 17: milvus.proto.query.ReleasePartitionsRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 18: milvus.proto.query.GetPartitionStatesRequest.base:type_name -> milvus.proto.common.MsgBase + 129, // 19: milvus.proto.query.GetPartitionStatesResponse.status:type_name -> milvus.proto.common.Status 43, // 20: milvus.proto.query.GetPartitionStatesResponse.partition_descriptions:type_name -> milvus.proto.query.PartitionStates - 123, // 21: milvus.proto.query.GetSegmentInfoRequest.base:type_name -> milvus.proto.common.MsgBase - 124, // 22: milvus.proto.query.GetSegmentInfoResponse.status:type_name -> milvus.proto.common.Status + 128, // 21: milvus.proto.query.GetSegmentInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 129, // 22: milvus.proto.query.GetSegmentInfoResponse.status:type_name -> milvus.proto.common.Status 44, // 23: milvus.proto.query.GetSegmentInfoResponse.infos:type_name -> milvus.proto.query.SegmentInfo - 123, // 24: milvus.proto.query.GetShardLeadersRequest.base:type_name -> milvus.proto.common.MsgBase - 124, // 25: milvus.proto.query.GetShardLeadersResponse.status:type_name -> milvus.proto.common.Status + 128, // 24: milvus.proto.query.GetShardLeadersRequest.base:type_name -> milvus.proto.common.MsgBase + 129, // 25: milvus.proto.query.GetShardLeadersResponse.status:type_name -> milvus.proto.common.Status 23, // 26: milvus.proto.query.GetShardLeadersResponse.shards:type_name -> milvus.proto.query.ShardLeadersList - 123, // 27: milvus.proto.query.UpdateResourceGroupsRequest.base:type_name -> milvus.proto.common.MsgBase - 104, // 28: milvus.proto.query.UpdateResourceGroupsRequest.resource_groups:type_name -> milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry - 123, // 29: milvus.proto.query.SyncNewCreatedPartitionRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 27: milvus.proto.query.UpdateResourceGroupsRequest.base:type_name -> milvus.proto.common.MsgBase + 109, // 28: milvus.proto.query.UpdateResourceGroupsRequest.resource_groups:type_name -> milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry + 128, // 29: milvus.proto.query.SyncNewCreatedPartitionRequest.base:type_name -> milvus.proto.common.MsgBase 4, // 30: milvus.proto.query.LoadMetaInfo.load_type:type_name -> milvus.proto.query.LoadType - 130, // 31: milvus.proto.query.LoadMetaInfo.db_properties:type_name -> milvus.proto.common.KeyValuePair - 123, // 32: milvus.proto.query.WatchDmChannelsRequest.base:type_name -> milvus.proto.common.MsgBase - 131, // 33: milvus.proto.query.WatchDmChannelsRequest.infos:type_name -> milvus.proto.data.VchannelInfo - 126, // 34: milvus.proto.query.WatchDmChannelsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 132, // 35: milvus.proto.query.WatchDmChannelsRequest.exclude_infos:type_name -> milvus.proto.data.SegmentInfo + 135, // 31: milvus.proto.query.LoadMetaInfo.db_properties:type_name -> milvus.proto.common.KeyValuePair + 128, // 32: milvus.proto.query.WatchDmChannelsRequest.base:type_name -> milvus.proto.common.MsgBase + 136, // 33: milvus.proto.query.WatchDmChannelsRequest.infos:type_name -> milvus.proto.data.VchannelInfo + 131, // 34: milvus.proto.query.WatchDmChannelsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 137, // 35: milvus.proto.query.WatchDmChannelsRequest.exclude_infos:type_name -> milvus.proto.data.SegmentInfo 25, // 36: milvus.proto.query.WatchDmChannelsRequest.load_meta:type_name -> milvus.proto.query.LoadMetaInfo - 105, // 37: milvus.proto.query.WatchDmChannelsRequest.segment_infos:type_name -> milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry - 129, // 38: milvus.proto.query.WatchDmChannelsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo - 106, // 39: milvus.proto.query.WatchDmChannelsRequest.sealed_segment_row_count:type_name -> milvus.proto.query.WatchDmChannelsRequest.SealedSegmentRowCountEntry - 123, // 40: milvus.proto.query.UnsubDmChannelRequest.base:type_name -> milvus.proto.common.MsgBase - 133, // 41: milvus.proto.query.SegmentLoadInfo.binlog_paths:type_name -> milvus.proto.data.FieldBinlog - 133, // 42: milvus.proto.query.SegmentLoadInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog - 133, // 43: milvus.proto.query.SegmentLoadInfo.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 110, // 37: milvus.proto.query.WatchDmChannelsRequest.segment_infos:type_name -> milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry + 134, // 38: milvus.proto.query.WatchDmChannelsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 111, // 39: milvus.proto.query.WatchDmChannelsRequest.sealed_segment_row_count:type_name -> milvus.proto.query.WatchDmChannelsRequest.SealedSegmentRowCountEntry + 128, // 40: milvus.proto.query.UnsubDmChannelRequest.base:type_name -> milvus.proto.common.MsgBase + 138, // 41: milvus.proto.query.SegmentLoadInfo.binlog_paths:type_name -> milvus.proto.data.FieldBinlog + 138, // 42: milvus.proto.query.SegmentLoadInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog + 138, // 43: milvus.proto.query.SegmentLoadInfo.deltalogs:type_name -> milvus.proto.data.FieldBinlog 29, // 44: milvus.proto.query.SegmentLoadInfo.index_infos:type_name -> milvus.proto.query.FieldIndexInfo - 134, // 45: milvus.proto.query.SegmentLoadInfo.start_position:type_name -> milvus.proto.msg.MsgPosition - 134, // 46: milvus.proto.query.SegmentLoadInfo.delta_position:type_name -> milvus.proto.msg.MsgPosition - 135, // 47: milvus.proto.query.SegmentLoadInfo.level:type_name -> milvus.proto.data.SegmentLevel - 107, // 48: milvus.proto.query.SegmentLoadInfo.textStatsLogs:type_name -> milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry - 133, // 49: milvus.proto.query.SegmentLoadInfo.bm25logs:type_name -> milvus.proto.data.FieldBinlog - 108, // 50: milvus.proto.query.SegmentLoadInfo.jsonKeyStatsLogs:type_name -> milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry - 127, // 51: milvus.proto.query.SegmentLoadInfo.priority:type_name -> milvus.proto.common.LoadPriority - 130, // 52: milvus.proto.query.FieldIndexInfo.index_params:type_name -> milvus.proto.common.KeyValuePair - 123, // 53: milvus.proto.query.LoadSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 139, // 45: milvus.proto.query.SegmentLoadInfo.start_position:type_name -> milvus.proto.msg.MsgPosition + 139, // 46: milvus.proto.query.SegmentLoadInfo.delta_position:type_name -> milvus.proto.msg.MsgPosition + 140, // 47: milvus.proto.query.SegmentLoadInfo.level:type_name -> milvus.proto.data.SegmentLevel + 112, // 48: milvus.proto.query.SegmentLoadInfo.textStatsLogs:type_name -> milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry + 138, // 49: milvus.proto.query.SegmentLoadInfo.bm25logs:type_name -> milvus.proto.data.FieldBinlog + 113, // 50: milvus.proto.query.SegmentLoadInfo.jsonKeyStatsLogs:type_name -> milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry + 132, // 51: milvus.proto.query.SegmentLoadInfo.priority:type_name -> milvus.proto.common.LoadPriority + 135, // 52: milvus.proto.query.FieldIndexInfo.index_params:type_name -> milvus.proto.common.KeyValuePair + 128, // 53: milvus.proto.query.LoadSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase 28, // 54: milvus.proto.query.LoadSegmentsRequest.infos:type_name -> milvus.proto.query.SegmentLoadInfo - 126, // 55: milvus.proto.query.LoadSegmentsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 131, // 55: milvus.proto.query.LoadSegmentsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema 25, // 56: milvus.proto.query.LoadSegmentsRequest.load_meta:type_name -> milvus.proto.query.LoadMetaInfo - 134, // 57: milvus.proto.query.LoadSegmentsRequest.delta_positions:type_name -> milvus.proto.msg.MsgPosition + 139, // 57: milvus.proto.query.LoadSegmentsRequest.delta_positions:type_name -> milvus.proto.msg.MsgPosition 0, // 58: milvus.proto.query.LoadSegmentsRequest.load_scope:type_name -> milvus.proto.query.LoadScope - 129, // 59: milvus.proto.query.LoadSegmentsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo - 123, // 60: milvus.proto.query.ReleaseSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 134, // 59: milvus.proto.query.LoadSegmentsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 128, // 60: milvus.proto.query.ReleaseSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase 1, // 61: milvus.proto.query.ReleaseSegmentsRequest.scope:type_name -> milvus.proto.query.DataScope - 134, // 62: milvus.proto.query.ReleaseSegmentsRequest.checkpoint:type_name -> milvus.proto.msg.MsgPosition - 136, // 63: milvus.proto.query.SearchRequest.req:type_name -> milvus.proto.internal.SearchRequest + 139, // 62: milvus.proto.query.ReleaseSegmentsRequest.checkpoint:type_name -> milvus.proto.msg.MsgPosition + 141, // 63: milvus.proto.query.SearchRequest.req:type_name -> milvus.proto.internal.SearchRequest 1, // 64: milvus.proto.query.SearchRequest.scope:type_name -> milvus.proto.query.DataScope - 137, // 65: milvus.proto.query.QueryRequest.req:type_name -> milvus.proto.internal.RetrieveRequest + 142, // 65: milvus.proto.query.QueryRequest.req:type_name -> milvus.proto.internal.RetrieveRequest 1, // 66: milvus.proto.query.QueryRequest.scope:type_name -> milvus.proto.query.DataScope - 123, // 67: milvus.proto.query.SyncReplicaSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 67: milvus.proto.query.SyncReplicaSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase 36, // 68: milvus.proto.query.SyncReplicaSegmentsRequest.replica_segments:type_name -> milvus.proto.query.ReplicaSegmentsInfo - 123, // 69: milvus.proto.query.GetLoadInfoRequest.base:type_name -> milvus.proto.common.MsgBase - 124, // 70: milvus.proto.query.GetLoadInfoResponse.status:type_name -> milvus.proto.common.Status - 126, // 71: milvus.proto.query.GetLoadInfoResponse.schema:type_name -> milvus.proto.schema.CollectionSchema + 128, // 69: milvus.proto.query.GetLoadInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 129, // 70: milvus.proto.query.GetLoadInfoResponse.status:type_name -> milvus.proto.common.Status + 131, // 71: milvus.proto.query.GetLoadInfoResponse.schema:type_name -> milvus.proto.schema.CollectionSchema 4, // 72: milvus.proto.query.GetLoadInfoResponse.load_type:type_name -> milvus.proto.query.LoadType - 123, // 73: milvus.proto.query.HandoffSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 73: milvus.proto.query.HandoffSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase 44, // 74: milvus.proto.query.HandoffSegmentsRequest.segmentInfos:type_name -> milvus.proto.query.SegmentInfo - 123, // 75: milvus.proto.query.LoadBalanceRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 75: milvus.proto.query.LoadBalanceRequest.base:type_name -> milvus.proto.common.MsgBase 3, // 76: milvus.proto.query.LoadBalanceRequest.balance_reason:type_name -> milvus.proto.query.TriggerCondition 44, // 77: milvus.proto.query.QueryChannelInfo.global_sealed_segments:type_name -> milvus.proto.query.SegmentInfo - 134, // 78: milvus.proto.query.QueryChannelInfo.seek_position:type_name -> milvus.proto.msg.MsgPosition + 139, // 78: milvus.proto.query.QueryChannelInfo.seek_position:type_name -> milvus.proto.msg.MsgPosition 2, // 79: milvus.proto.query.PartitionStates.state:type_name -> milvus.proto.query.PartitionState - 138, // 80: milvus.proto.query.SegmentInfo.segment_state:type_name -> milvus.proto.common.SegmentState + 143, // 80: milvus.proto.query.SegmentInfo.segment_state:type_name -> milvus.proto.common.SegmentState 29, // 81: milvus.proto.query.SegmentInfo.index_infos:type_name -> milvus.proto.query.FieldIndexInfo - 135, // 82: milvus.proto.query.SegmentInfo.level:type_name -> milvus.proto.data.SegmentLevel + 140, // 82: milvus.proto.query.SegmentInfo.level:type_name -> milvus.proto.data.SegmentLevel 43, // 83: milvus.proto.query.CollectionInfo.partition_states:type_name -> milvus.proto.query.PartitionStates 4, // 84: milvus.proto.query.CollectionInfo.load_type:type_name -> milvus.proto.query.LoadType - 126, // 85: milvus.proto.query.CollectionInfo.schema:type_name -> milvus.proto.schema.CollectionSchema + 131, // 85: milvus.proto.query.CollectionInfo.schema:type_name -> milvus.proto.schema.CollectionSchema 46, // 86: milvus.proto.query.UnsubscribeChannelInfo.collection_channels:type_name -> milvus.proto.query.UnsubscribeChannels 44, // 87: milvus.proto.query.SegmentChangeInfo.online_segments:type_name -> milvus.proto.query.SegmentInfo 44, // 88: milvus.proto.query.SegmentChangeInfo.offline_segments:type_name -> milvus.proto.query.SegmentInfo - 123, // 89: milvus.proto.query.SealedSegmentsChangeInfo.base:type_name -> milvus.proto.common.MsgBase + 128, // 89: milvus.proto.query.SealedSegmentsChangeInfo.base:type_name -> milvus.proto.common.MsgBase 48, // 90: milvus.proto.query.SealedSegmentsChangeInfo.infos:type_name -> milvus.proto.query.SegmentChangeInfo - 123, // 91: milvus.proto.query.GetDataDistributionRequest.base:type_name -> milvus.proto.common.MsgBase - 109, // 92: milvus.proto.query.GetDataDistributionRequest.checkpoints:type_name -> milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry - 124, // 93: milvus.proto.query.GetDataDistributionResponse.status:type_name -> milvus.proto.common.Status + 128, // 91: milvus.proto.query.GetDataDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 114, // 92: milvus.proto.query.GetDataDistributionRequest.checkpoints:type_name -> milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry + 129, // 93: milvus.proto.query.GetDataDistributionResponse.status:type_name -> milvus.proto.common.Status 55, // 94: milvus.proto.query.GetDataDistributionResponse.segments:type_name -> milvus.proto.query.SegmentVersionInfo 56, // 95: milvus.proto.query.GetDataDistributionResponse.channels:type_name -> milvus.proto.query.ChannelVersionInfo 52, // 96: milvus.proto.query.GetDataDistributionResponse.leader_views:type_name -> milvus.proto.query.LeaderView - 110, // 97: milvus.proto.query.LeaderView.segment_dist:type_name -> milvus.proto.query.LeaderView.SegmentDistEntry - 111, // 98: milvus.proto.query.LeaderView.growing_segments:type_name -> milvus.proto.query.LeaderView.GrowingSegmentsEntry - 112, // 99: milvus.proto.query.LeaderView.partition_stats_versions:type_name -> milvus.proto.query.LeaderView.PartitionStatsVersionsEntry + 115, // 97: milvus.proto.query.LeaderView.segment_dist:type_name -> milvus.proto.query.LeaderView.SegmentDistEntry + 116, // 98: milvus.proto.query.LeaderView.growing_segments:type_name -> milvus.proto.query.LeaderView.GrowingSegmentsEntry + 117, // 99: milvus.proto.query.LeaderView.partition_stats_versions:type_name -> milvus.proto.query.LeaderView.PartitionStatsVersionsEntry 53, // 100: milvus.proto.query.LeaderView.status:type_name -> milvus.proto.query.LeaderViewStatus - 113, // 101: milvus.proto.query.SegmentVersionInfo.index_info:type_name -> milvus.proto.query.SegmentVersionInfo.IndexInfoEntry - 135, // 102: milvus.proto.query.SegmentVersionInfo.level:type_name -> milvus.proto.data.SegmentLevel - 114, // 103: milvus.proto.query.SegmentVersionInfo.json_stats_info:type_name -> milvus.proto.query.SegmentVersionInfo.JsonStatsInfoEntry + 118, // 101: milvus.proto.query.SegmentVersionInfo.index_info:type_name -> milvus.proto.query.SegmentVersionInfo.IndexInfoEntry + 140, // 102: milvus.proto.query.SegmentVersionInfo.level:type_name -> milvus.proto.data.SegmentLevel + 119, // 103: milvus.proto.query.SegmentVersionInfo.json_stats_info:type_name -> milvus.proto.query.SegmentVersionInfo.JsonStatsInfoEntry 5, // 104: milvus.proto.query.CollectionLoadInfo.status:type_name -> milvus.proto.query.LoadStatus - 115, // 105: milvus.proto.query.CollectionLoadInfo.field_indexID:type_name -> milvus.proto.query.CollectionLoadInfo.FieldIndexIDEntry + 120, // 105: milvus.proto.query.CollectionLoadInfo.field_indexID:type_name -> milvus.proto.query.CollectionLoadInfo.FieldIndexIDEntry 4, // 106: milvus.proto.query.CollectionLoadInfo.load_type:type_name -> milvus.proto.query.LoadType 5, // 107: milvus.proto.query.PartitionLoadInfo.status:type_name -> milvus.proto.query.LoadStatus - 116, // 108: milvus.proto.query.PartitionLoadInfo.field_indexID:type_name -> milvus.proto.query.PartitionLoadInfo.FieldIndexIDEntry - 117, // 109: milvus.proto.query.Replica.channel_node_infos:type_name -> milvus.proto.query.Replica.ChannelNodeInfosEntry + 121, // 108: milvus.proto.query.PartitionLoadInfo.field_indexID:type_name -> milvus.proto.query.PartitionLoadInfo.FieldIndexIDEntry + 122, // 109: milvus.proto.query.Replica.channel_node_infos:type_name -> milvus.proto.query.Replica.ChannelNodeInfosEntry 6, // 110: milvus.proto.query.SyncAction.type:type_name -> milvus.proto.query.SyncType 28, // 111: milvus.proto.query.SyncAction.info:type_name -> milvus.proto.query.SegmentLoadInfo - 134, // 112: milvus.proto.query.SyncAction.checkpoint:type_name -> milvus.proto.msg.MsgPosition - 118, // 113: milvus.proto.query.SyncAction.partition_stats_versions:type_name -> milvus.proto.query.SyncAction.PartitionStatsVersionsEntry - 134, // 114: milvus.proto.query.SyncAction.deleteCP:type_name -> milvus.proto.msg.MsgPosition - 119, // 115: milvus.proto.query.SyncAction.sealed_segment_row_count:type_name -> milvus.proto.query.SyncAction.SealedSegmentRowCountEntry - 123, // 116: milvus.proto.query.SyncDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 139, // 112: milvus.proto.query.SyncAction.checkpoint:type_name -> milvus.proto.msg.MsgPosition + 123, // 113: milvus.proto.query.SyncAction.partition_stats_versions:type_name -> milvus.proto.query.SyncAction.PartitionStatsVersionsEntry + 139, // 114: milvus.proto.query.SyncAction.deleteCP:type_name -> milvus.proto.msg.MsgPosition + 124, // 115: milvus.proto.query.SyncAction.sealed_segment_row_count:type_name -> milvus.proto.query.SyncAction.SealedSegmentRowCountEntry + 128, // 116: milvus.proto.query.SyncDistributionRequest.base:type_name -> milvus.proto.common.MsgBase 61, // 117: milvus.proto.query.SyncDistributionRequest.actions:type_name -> milvus.proto.query.SyncAction - 126, // 118: milvus.proto.query.SyncDistributionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 131, // 118: milvus.proto.query.SyncDistributionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema 25, // 119: milvus.proto.query.SyncDistributionRequest.load_meta:type_name -> milvus.proto.query.LoadMetaInfo - 129, // 120: milvus.proto.query.SyncDistributionRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo - 139, // 121: milvus.proto.query.ResourceGroup.config:type_name -> milvus.proto.rg.ResourceGroupConfig - 123, // 122: milvus.proto.query.TransferReplicaRequest.base:type_name -> milvus.proto.common.MsgBase - 123, // 123: milvus.proto.query.DescribeResourceGroupRequest.base:type_name -> milvus.proto.common.MsgBase - 124, // 124: milvus.proto.query.DescribeResourceGroupResponse.status:type_name -> milvus.proto.common.Status + 134, // 120: milvus.proto.query.SyncDistributionRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 144, // 121: milvus.proto.query.ResourceGroup.config:type_name -> milvus.proto.rg.ResourceGroupConfig + 128, // 122: milvus.proto.query.TransferReplicaRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 123: milvus.proto.query.DescribeResourceGroupRequest.base:type_name -> milvus.proto.common.MsgBase + 129, // 124: milvus.proto.query.DescribeResourceGroupResponse.status:type_name -> milvus.proto.common.Status 67, // 125: milvus.proto.query.DescribeResourceGroupResponse.resource_group:type_name -> milvus.proto.query.ResourceGroupInfo - 120, // 126: milvus.proto.query.ResourceGroupInfo.num_loaded_replica:type_name -> milvus.proto.query.ResourceGroupInfo.NumLoadedReplicaEntry - 121, // 127: milvus.proto.query.ResourceGroupInfo.num_outgoing_node:type_name -> milvus.proto.query.ResourceGroupInfo.NumOutgoingNodeEntry - 122, // 128: milvus.proto.query.ResourceGroupInfo.num_incoming_node:type_name -> milvus.proto.query.ResourceGroupInfo.NumIncomingNodeEntry - 139, // 129: milvus.proto.query.ResourceGroupInfo.config:type_name -> milvus.proto.rg.ResourceGroupConfig - 140, // 130: milvus.proto.query.ResourceGroupInfo.nodes:type_name -> milvus.proto.common.NodeInfo - 123, // 131: milvus.proto.query.DeleteRequest.base:type_name -> milvus.proto.common.MsgBase - 141, // 132: milvus.proto.query.DeleteRequest.primary_keys:type_name -> milvus.proto.schema.IDs + 125, // 126: milvus.proto.query.ResourceGroupInfo.num_loaded_replica:type_name -> milvus.proto.query.ResourceGroupInfo.NumLoadedReplicaEntry + 126, // 127: milvus.proto.query.ResourceGroupInfo.num_outgoing_node:type_name -> milvus.proto.query.ResourceGroupInfo.NumOutgoingNodeEntry + 127, // 128: milvus.proto.query.ResourceGroupInfo.num_incoming_node:type_name -> milvus.proto.query.ResourceGroupInfo.NumIncomingNodeEntry + 144, // 129: milvus.proto.query.ResourceGroupInfo.config:type_name -> milvus.proto.rg.ResourceGroupConfig + 145, // 130: milvus.proto.query.ResourceGroupInfo.nodes:type_name -> milvus.proto.common.NodeInfo + 128, // 131: milvus.proto.query.DeleteRequest.base:type_name -> milvus.proto.common.MsgBase + 146, // 132: milvus.proto.query.DeleteRequest.primary_keys:type_name -> milvus.proto.schema.IDs 1, // 133: milvus.proto.query.DeleteRequest.scope:type_name -> milvus.proto.query.DataScope - 123, // 134: milvus.proto.query.DeleteBatchRequest.base:type_name -> milvus.proto.common.MsgBase - 141, // 135: milvus.proto.query.DeleteBatchRequest.primary_keys:type_name -> milvus.proto.schema.IDs + 128, // 134: milvus.proto.query.DeleteBatchRequest.base:type_name -> milvus.proto.common.MsgBase + 146, // 135: milvus.proto.query.DeleteBatchRequest.primary_keys:type_name -> milvus.proto.schema.IDs 1, // 136: milvus.proto.query.DeleteBatchRequest.scope:type_name -> milvus.proto.query.DataScope - 124, // 137: milvus.proto.query.DeleteBatchResponse.status:type_name -> milvus.proto.common.Status - 123, // 138: milvus.proto.query.ActivateCheckerRequest.base:type_name -> milvus.proto.common.MsgBase - 123, // 139: milvus.proto.query.DeactivateCheckerRequest.base:type_name -> milvus.proto.common.MsgBase - 123, // 140: milvus.proto.query.ListCheckersRequest.base:type_name -> milvus.proto.common.MsgBase - 124, // 141: milvus.proto.query.ListCheckersResponse.status:type_name -> milvus.proto.common.Status + 129, // 137: milvus.proto.query.DeleteBatchResponse.status:type_name -> milvus.proto.common.Status + 128, // 138: milvus.proto.query.ActivateCheckerRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 139: milvus.proto.query.DeactivateCheckerRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 140: milvus.proto.query.ListCheckersRequest.base:type_name -> milvus.proto.common.MsgBase + 129, // 141: milvus.proto.query.ListCheckersResponse.status:type_name -> milvus.proto.common.Status 75, // 142: milvus.proto.query.ListCheckersResponse.checkerInfos:type_name -> milvus.proto.query.CheckerInfo - 135, // 143: milvus.proto.query.SegmentTarget.level:type_name -> milvus.proto.data.SegmentLevel + 140, // 143: milvus.proto.query.SegmentTarget.level:type_name -> milvus.proto.data.SegmentLevel 76, // 144: milvus.proto.query.PartitionTarget.segments:type_name -> milvus.proto.query.SegmentTarget 77, // 145: milvus.proto.query.ChannelTarget.partition_targets:type_name -> milvus.proto.query.PartitionTarget - 134, // 146: milvus.proto.query.ChannelTarget.seek_position:type_name -> milvus.proto.msg.MsgPosition - 134, // 147: milvus.proto.query.ChannelTarget.delete_checkpoint:type_name -> milvus.proto.msg.MsgPosition + 139, // 146: milvus.proto.query.ChannelTarget.seek_position:type_name -> milvus.proto.msg.MsgPosition + 139, // 147: milvus.proto.query.ChannelTarget.delete_checkpoint:type_name -> milvus.proto.msg.MsgPosition 78, // 148: milvus.proto.query.CollectionTarget.Channel_targets:type_name -> milvus.proto.query.ChannelTarget - 123, // 149: milvus.proto.query.ListQueryNodeRequest.base:type_name -> milvus.proto.common.MsgBase - 124, // 150: milvus.proto.query.ListQueryNodeResponse.status:type_name -> milvus.proto.common.Status + 128, // 149: milvus.proto.query.ListQueryNodeRequest.base:type_name -> milvus.proto.common.MsgBase + 129, // 150: milvus.proto.query.ListQueryNodeResponse.status:type_name -> milvus.proto.common.Status 80, // 151: milvus.proto.query.ListQueryNodeResponse.nodeInfos:type_name -> milvus.proto.query.NodeInfo - 123, // 152: milvus.proto.query.GetQueryNodeDistributionRequest.base:type_name -> milvus.proto.common.MsgBase - 124, // 153: milvus.proto.query.GetQueryNodeDistributionResponse.status:type_name -> milvus.proto.common.Status - 123, // 154: milvus.proto.query.SuspendBalanceRequest.base:type_name -> milvus.proto.common.MsgBase - 123, // 155: milvus.proto.query.ResumeBalanceRequest.base:type_name -> milvus.proto.common.MsgBase - 123, // 156: milvus.proto.query.CheckBalanceStatusRequest.base:type_name -> milvus.proto.common.MsgBase - 124, // 157: milvus.proto.query.CheckBalanceStatusResponse.status:type_name -> milvus.proto.common.Status - 123, // 158: milvus.proto.query.SuspendNodeRequest.base:type_name -> milvus.proto.common.MsgBase - 123, // 159: milvus.proto.query.ResumeNodeRequest.base:type_name -> milvus.proto.common.MsgBase - 123, // 160: milvus.proto.query.TransferSegmentRequest.base:type_name -> milvus.proto.common.MsgBase - 123, // 161: milvus.proto.query.TransferChannelRequest.base:type_name -> milvus.proto.common.MsgBase - 123, // 162: milvus.proto.query.CheckQueryNodeDistributionRequest.base:type_name -> milvus.proto.common.MsgBase - 123, // 163: milvus.proto.query.UpdateLoadConfigRequest.base:type_name -> milvus.proto.common.MsgBase - 123, // 164: milvus.proto.query.UpdateSchemaRequest.base:type_name -> milvus.proto.common.MsgBase - 126, // 165: milvus.proto.query.UpdateSchemaRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 123, // 166: milvus.proto.query.RunAnalyzerRequest.base:type_name -> milvus.proto.common.MsgBase - 123, // 167: milvus.proto.query.ValidateAnalyzerRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 152: milvus.proto.query.GetQueryNodeDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 129, // 153: milvus.proto.query.GetQueryNodeDistributionResponse.status:type_name -> milvus.proto.common.Status + 128, // 154: milvus.proto.query.SuspendBalanceRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 155: milvus.proto.query.ResumeBalanceRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 156: milvus.proto.query.CheckBalanceStatusRequest.base:type_name -> milvus.proto.common.MsgBase + 129, // 157: milvus.proto.query.CheckBalanceStatusResponse.status:type_name -> milvus.proto.common.Status + 128, // 158: milvus.proto.query.SuspendNodeRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 159: milvus.proto.query.ResumeNodeRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 160: milvus.proto.query.TransferSegmentRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 161: milvus.proto.query.TransferChannelRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 162: milvus.proto.query.CheckQueryNodeDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 163: milvus.proto.query.UpdateLoadConfigRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 164: milvus.proto.query.UpdateSchemaRequest.base:type_name -> milvus.proto.common.MsgBase + 131, // 165: milvus.proto.query.UpdateSchemaRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 128, // 166: milvus.proto.query.RunAnalyzerRequest.base:type_name -> milvus.proto.common.MsgBase + 128, // 167: milvus.proto.query.ValidateAnalyzerRequest.base:type_name -> milvus.proto.common.MsgBase 97, // 168: milvus.proto.query.ValidateAnalyzerRequest.analyzer_infos:type_name -> milvus.proto.query.AnalyzerInfo - 123, // 169: milvus.proto.query.ListLoadedSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase - 124, // 170: milvus.proto.query.ListLoadedSegmentsResponse.status:type_name -> milvus.proto.common.Status - 123, // 171: milvus.proto.query.DropIndexRequest.base:type_name -> milvus.proto.common.MsgBase - 139, // 172: milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry.value:type_name -> milvus.proto.rg.ResourceGroupConfig - 132, // 173: milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry.value:type_name -> milvus.proto.data.SegmentInfo - 142, // 174: milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats - 143, // 175: milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry.value:type_name -> milvus.proto.data.JsonKeyStats - 134, // 176: milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry.value:type_name -> milvus.proto.msg.MsgPosition - 54, // 177: milvus.proto.query.LeaderView.SegmentDistEntry.value:type_name -> milvus.proto.query.SegmentDist - 134, // 178: milvus.proto.query.LeaderView.GrowingSegmentsEntry.value:type_name -> milvus.proto.msg.MsgPosition - 29, // 179: milvus.proto.query.SegmentVersionInfo.IndexInfoEntry.value:type_name -> milvus.proto.query.FieldIndexInfo - 30, // 180: milvus.proto.query.SegmentVersionInfo.JsonStatsInfoEntry.value:type_name -> milvus.proto.query.JsonStatsInfo - 59, // 181: milvus.proto.query.Replica.ChannelNodeInfosEntry.value:type_name -> milvus.proto.query.ChannelNodeInfo - 7, // 182: milvus.proto.query.QueryCoord.ShowLoadCollections:input_type -> milvus.proto.query.ShowCollectionsRequest - 9, // 183: milvus.proto.query.QueryCoord.ShowLoadPartitions:input_type -> milvus.proto.query.ShowPartitionsRequest - 14, // 184: milvus.proto.query.QueryCoord.LoadPartitions:input_type -> milvus.proto.query.LoadPartitionsRequest - 15, // 185: milvus.proto.query.QueryCoord.ReleasePartitions:input_type -> milvus.proto.query.ReleasePartitionsRequest - 11, // 186: milvus.proto.query.QueryCoord.LoadCollection:input_type -> milvus.proto.query.LoadCollectionRequest - 12, // 187: milvus.proto.query.QueryCoord.ReleaseCollection:input_type -> milvus.proto.query.ReleaseCollectionRequest - 24, // 188: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:input_type -> milvus.proto.query.SyncNewCreatedPartitionRequest - 16, // 189: milvus.proto.query.QueryCoord.GetPartitionStates:input_type -> milvus.proto.query.GetPartitionStatesRequest - 18, // 190: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:input_type -> milvus.proto.query.GetSegmentInfoRequest - 40, // 191: milvus.proto.query.QueryCoord.LoadBalance:input_type -> milvus.proto.query.LoadBalanceRequest - 144, // 192: milvus.proto.query.QueryCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest - 145, // 193: milvus.proto.query.QueryCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest - 146, // 194: milvus.proto.query.QueryCoord.GetReplicas:input_type -> milvus.proto.milvus.GetReplicasRequest - 20, // 195: milvus.proto.query.QueryCoord.GetShardLeaders:input_type -> milvus.proto.query.GetShardLeadersRequest - 147, // 196: milvus.proto.query.QueryCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest - 148, // 197: milvus.proto.query.QueryCoord.CreateResourceGroup:input_type -> milvus.proto.milvus.CreateResourceGroupRequest - 22, // 198: milvus.proto.query.QueryCoord.UpdateResourceGroups:input_type -> milvus.proto.query.UpdateResourceGroupsRequest - 149, // 199: milvus.proto.query.QueryCoord.DropResourceGroup:input_type -> milvus.proto.milvus.DropResourceGroupRequest - 150, // 200: milvus.proto.query.QueryCoord.TransferNode:input_type -> milvus.proto.milvus.TransferNodeRequest - 64, // 201: milvus.proto.query.QueryCoord.TransferReplica:input_type -> milvus.proto.query.TransferReplicaRequest - 151, // 202: milvus.proto.query.QueryCoord.ListResourceGroups:input_type -> milvus.proto.milvus.ListResourceGroupsRequest - 65, // 203: milvus.proto.query.QueryCoord.DescribeResourceGroup:input_type -> milvus.proto.query.DescribeResourceGroupRequest - 99, // 204: milvus.proto.query.QueryCoord.ListLoadedSegments:input_type -> milvus.proto.query.ListLoadedSegmentsRequest - 73, // 205: milvus.proto.query.QueryCoord.ListCheckers:input_type -> milvus.proto.query.ListCheckersRequest - 71, // 206: milvus.proto.query.QueryCoord.ActivateChecker:input_type -> milvus.proto.query.ActivateCheckerRequest - 72, // 207: milvus.proto.query.QueryCoord.DeactivateChecker:input_type -> milvus.proto.query.DeactivateCheckerRequest - 81, // 208: milvus.proto.query.QueryCoord.ListQueryNode:input_type -> milvus.proto.query.ListQueryNodeRequest - 83, // 209: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:input_type -> milvus.proto.query.GetQueryNodeDistributionRequest - 85, // 210: milvus.proto.query.QueryCoord.SuspendBalance:input_type -> milvus.proto.query.SuspendBalanceRequest - 86, // 211: milvus.proto.query.QueryCoord.ResumeBalance:input_type -> milvus.proto.query.ResumeBalanceRequest - 87, // 212: milvus.proto.query.QueryCoord.CheckBalanceStatus:input_type -> milvus.proto.query.CheckBalanceStatusRequest - 89, // 213: milvus.proto.query.QueryCoord.SuspendNode:input_type -> milvus.proto.query.SuspendNodeRequest - 90, // 214: milvus.proto.query.QueryCoord.ResumeNode:input_type -> milvus.proto.query.ResumeNodeRequest - 91, // 215: milvus.proto.query.QueryCoord.TransferSegment:input_type -> milvus.proto.query.TransferSegmentRequest - 92, // 216: milvus.proto.query.QueryCoord.TransferChannel:input_type -> milvus.proto.query.TransferChannelRequest - 93, // 217: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:input_type -> milvus.proto.query.CheckQueryNodeDistributionRequest - 94, // 218: milvus.proto.query.QueryCoord.UpdateLoadConfig:input_type -> milvus.proto.query.UpdateLoadConfigRequest - 96, // 219: milvus.proto.query.QueryCoord.RunAnalyzer:input_type -> milvus.proto.query.RunAnalyzerRequest - 98, // 220: milvus.proto.query.QueryCoord.ValidateAnalyzer:input_type -> milvus.proto.query.ValidateAnalyzerRequest - 152, // 221: milvus.proto.query.QueryNode.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest - 153, // 222: milvus.proto.query.QueryNode.GetTimeTickChannel:input_type -> milvus.proto.internal.GetTimeTickChannelRequest - 154, // 223: milvus.proto.query.QueryNode.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest - 26, // 224: milvus.proto.query.QueryNode.WatchDmChannels:input_type -> milvus.proto.query.WatchDmChannelsRequest - 27, // 225: milvus.proto.query.QueryNode.UnsubDmChannel:input_type -> milvus.proto.query.UnsubDmChannelRequest - 31, // 226: milvus.proto.query.QueryNode.LoadSegments:input_type -> milvus.proto.query.LoadSegmentsRequest - 12, // 227: milvus.proto.query.QueryNode.ReleaseCollection:input_type -> milvus.proto.query.ReleaseCollectionRequest - 14, // 228: milvus.proto.query.QueryNode.LoadPartitions:input_type -> milvus.proto.query.LoadPartitionsRequest - 15, // 229: milvus.proto.query.QueryNode.ReleasePartitions:input_type -> milvus.proto.query.ReleasePartitionsRequest - 32, // 230: milvus.proto.query.QueryNode.ReleaseSegments:input_type -> milvus.proto.query.ReleaseSegmentsRequest - 18, // 231: milvus.proto.query.QueryNode.GetSegmentInfo:input_type -> milvus.proto.query.GetSegmentInfoRequest - 35, // 232: milvus.proto.query.QueryNode.SyncReplicaSegments:input_type -> milvus.proto.query.SyncReplicaSegmentsRequest - 13, // 233: milvus.proto.query.QueryNode.GetStatistics:input_type -> milvus.proto.query.GetStatisticsRequest - 33, // 234: milvus.proto.query.QueryNode.Search:input_type -> milvus.proto.query.SearchRequest - 33, // 235: milvus.proto.query.QueryNode.SearchSegments:input_type -> milvus.proto.query.SearchRequest - 34, // 236: milvus.proto.query.QueryNode.Query:input_type -> milvus.proto.query.QueryRequest - 34, // 237: milvus.proto.query.QueryNode.QueryStream:input_type -> milvus.proto.query.QueryRequest - 34, // 238: milvus.proto.query.QueryNode.QuerySegments:input_type -> milvus.proto.query.QueryRequest - 34, // 239: milvus.proto.query.QueryNode.QueryStreamSegments:input_type -> milvus.proto.query.QueryRequest - 144, // 240: milvus.proto.query.QueryNode.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest - 145, // 241: milvus.proto.query.QueryNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest - 50, // 242: milvus.proto.query.QueryNode.GetDataDistribution:input_type -> milvus.proto.query.GetDataDistributionRequest - 62, // 243: milvus.proto.query.QueryNode.SyncDistribution:input_type -> milvus.proto.query.SyncDistributionRequest - 68, // 244: milvus.proto.query.QueryNode.Delete:input_type -> milvus.proto.query.DeleteRequest - 69, // 245: milvus.proto.query.QueryNode.DeleteBatch:input_type -> milvus.proto.query.DeleteBatchRequest - 95, // 246: milvus.proto.query.QueryNode.UpdateSchema:input_type -> milvus.proto.query.UpdateSchemaRequest - 96, // 247: milvus.proto.query.QueryNode.RunAnalyzer:input_type -> milvus.proto.query.RunAnalyzerRequest - 101, // 248: milvus.proto.query.QueryNode.DropIndex:input_type -> milvus.proto.query.DropIndexRequest - 98, // 249: milvus.proto.query.QueryNode.ValidateAnalyzer:input_type -> milvus.proto.query.ValidateAnalyzerRequest - 8, // 250: milvus.proto.query.QueryCoord.ShowLoadCollections:output_type -> milvus.proto.query.ShowCollectionsResponse - 10, // 251: milvus.proto.query.QueryCoord.ShowLoadPartitions:output_type -> milvus.proto.query.ShowPartitionsResponse - 124, // 252: milvus.proto.query.QueryCoord.LoadPartitions:output_type -> milvus.proto.common.Status - 124, // 253: milvus.proto.query.QueryCoord.ReleasePartitions:output_type -> milvus.proto.common.Status - 124, // 254: milvus.proto.query.QueryCoord.LoadCollection:output_type -> milvus.proto.common.Status - 124, // 255: milvus.proto.query.QueryCoord.ReleaseCollection:output_type -> milvus.proto.common.Status - 124, // 256: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:output_type -> milvus.proto.common.Status - 17, // 257: milvus.proto.query.QueryCoord.GetPartitionStates:output_type -> milvus.proto.query.GetPartitionStatesResponse - 19, // 258: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse - 124, // 259: milvus.proto.query.QueryCoord.LoadBalance:output_type -> milvus.proto.common.Status - 155, // 260: milvus.proto.query.QueryCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 156, // 261: milvus.proto.query.QueryCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 157, // 262: milvus.proto.query.QueryCoord.GetReplicas:output_type -> milvus.proto.milvus.GetReplicasResponse - 21, // 263: milvus.proto.query.QueryCoord.GetShardLeaders:output_type -> milvus.proto.query.GetShardLeadersResponse - 158, // 264: milvus.proto.query.QueryCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse - 124, // 265: milvus.proto.query.QueryCoord.CreateResourceGroup:output_type -> milvus.proto.common.Status - 124, // 266: milvus.proto.query.QueryCoord.UpdateResourceGroups:output_type -> milvus.proto.common.Status - 124, // 267: milvus.proto.query.QueryCoord.DropResourceGroup:output_type -> milvus.proto.common.Status - 124, // 268: milvus.proto.query.QueryCoord.TransferNode:output_type -> milvus.proto.common.Status - 124, // 269: milvus.proto.query.QueryCoord.TransferReplica:output_type -> milvus.proto.common.Status - 159, // 270: milvus.proto.query.QueryCoord.ListResourceGroups:output_type -> milvus.proto.milvus.ListResourceGroupsResponse - 66, // 271: milvus.proto.query.QueryCoord.DescribeResourceGroup:output_type -> milvus.proto.query.DescribeResourceGroupResponse - 100, // 272: milvus.proto.query.QueryCoord.ListLoadedSegments:output_type -> milvus.proto.query.ListLoadedSegmentsResponse - 74, // 273: milvus.proto.query.QueryCoord.ListCheckers:output_type -> milvus.proto.query.ListCheckersResponse - 124, // 274: milvus.proto.query.QueryCoord.ActivateChecker:output_type -> milvus.proto.common.Status - 124, // 275: milvus.proto.query.QueryCoord.DeactivateChecker:output_type -> milvus.proto.common.Status - 82, // 276: milvus.proto.query.QueryCoord.ListQueryNode:output_type -> milvus.proto.query.ListQueryNodeResponse - 84, // 277: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:output_type -> milvus.proto.query.GetQueryNodeDistributionResponse - 124, // 278: milvus.proto.query.QueryCoord.SuspendBalance:output_type -> milvus.proto.common.Status - 124, // 279: milvus.proto.query.QueryCoord.ResumeBalance:output_type -> milvus.proto.common.Status - 88, // 280: milvus.proto.query.QueryCoord.CheckBalanceStatus:output_type -> milvus.proto.query.CheckBalanceStatusResponse - 124, // 281: milvus.proto.query.QueryCoord.SuspendNode:output_type -> milvus.proto.common.Status - 124, // 282: milvus.proto.query.QueryCoord.ResumeNode:output_type -> milvus.proto.common.Status - 124, // 283: milvus.proto.query.QueryCoord.TransferSegment:output_type -> milvus.proto.common.Status - 124, // 284: milvus.proto.query.QueryCoord.TransferChannel:output_type -> milvus.proto.common.Status - 124, // 285: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:output_type -> milvus.proto.common.Status - 124, // 286: milvus.proto.query.QueryCoord.UpdateLoadConfig:output_type -> milvus.proto.common.Status - 160, // 287: milvus.proto.query.QueryCoord.RunAnalyzer:output_type -> milvus.proto.milvus.RunAnalyzerResponse - 124, // 288: milvus.proto.query.QueryCoord.ValidateAnalyzer:output_type -> milvus.proto.common.Status - 161, // 289: milvus.proto.query.QueryNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates - 162, // 290: milvus.proto.query.QueryNode.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse - 162, // 291: milvus.proto.query.QueryNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse - 124, // 292: milvus.proto.query.QueryNode.WatchDmChannels:output_type -> milvus.proto.common.Status - 124, // 293: milvus.proto.query.QueryNode.UnsubDmChannel:output_type -> milvus.proto.common.Status - 124, // 294: milvus.proto.query.QueryNode.LoadSegments:output_type -> milvus.proto.common.Status - 124, // 295: milvus.proto.query.QueryNode.ReleaseCollection:output_type -> milvus.proto.common.Status - 124, // 296: milvus.proto.query.QueryNode.LoadPartitions:output_type -> milvus.proto.common.Status - 124, // 297: milvus.proto.query.QueryNode.ReleasePartitions:output_type -> milvus.proto.common.Status - 124, // 298: milvus.proto.query.QueryNode.ReleaseSegments:output_type -> milvus.proto.common.Status - 19, // 299: milvus.proto.query.QueryNode.GetSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse - 124, // 300: milvus.proto.query.QueryNode.SyncReplicaSegments:output_type -> milvus.proto.common.Status - 163, // 301: milvus.proto.query.QueryNode.GetStatistics:output_type -> milvus.proto.internal.GetStatisticsResponse - 164, // 302: milvus.proto.query.QueryNode.Search:output_type -> milvus.proto.internal.SearchResults - 164, // 303: milvus.proto.query.QueryNode.SearchSegments:output_type -> milvus.proto.internal.SearchResults - 165, // 304: milvus.proto.query.QueryNode.Query:output_type -> milvus.proto.internal.RetrieveResults - 165, // 305: milvus.proto.query.QueryNode.QueryStream:output_type -> milvus.proto.internal.RetrieveResults - 165, // 306: milvus.proto.query.QueryNode.QuerySegments:output_type -> milvus.proto.internal.RetrieveResults - 165, // 307: milvus.proto.query.QueryNode.QueryStreamSegments:output_type -> milvus.proto.internal.RetrieveResults - 155, // 308: milvus.proto.query.QueryNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 156, // 309: milvus.proto.query.QueryNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 51, // 310: milvus.proto.query.QueryNode.GetDataDistribution:output_type -> milvus.proto.query.GetDataDistributionResponse - 124, // 311: milvus.proto.query.QueryNode.SyncDistribution:output_type -> milvus.proto.common.Status - 124, // 312: milvus.proto.query.QueryNode.Delete:output_type -> milvus.proto.common.Status - 70, // 313: milvus.proto.query.QueryNode.DeleteBatch:output_type -> milvus.proto.query.DeleteBatchResponse - 124, // 314: milvus.proto.query.QueryNode.UpdateSchema:output_type -> milvus.proto.common.Status - 160, // 315: milvus.proto.query.QueryNode.RunAnalyzer:output_type -> milvus.proto.milvus.RunAnalyzerResponse - 124, // 316: milvus.proto.query.QueryNode.DropIndex:output_type -> milvus.proto.common.Status - 124, // 317: milvus.proto.query.QueryNode.ValidateAnalyzer:output_type -> milvus.proto.common.Status - 250, // [250:318] is the sub-list for method output_type - 182, // [182:250] is the sub-list for method input_type - 182, // [182:182] is the sub-list for extension type_name - 182, // [182:182] is the sub-list for extension extendee - 0, // [0:182] is the sub-list for field type_name + 128, // 169: milvus.proto.query.GetHighlightRequest.base:type_name -> milvus.proto.common.MsgBase + 99, // 170: milvus.proto.query.GetHighlightRequest.tasks:type_name -> milvus.proto.query.HighlightTask + 101, // 171: milvus.proto.query.HighlightResult.fragments:type_name -> milvus.proto.query.HighlightFragment + 129, // 172: milvus.proto.query.GetHighlightResponse.status:type_name -> milvus.proto.common.Status + 102, // 173: milvus.proto.query.GetHighlightResponse.results:type_name -> milvus.proto.query.HighlightResult + 128, // 174: milvus.proto.query.ListLoadedSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 129, // 175: milvus.proto.query.ListLoadedSegmentsResponse.status:type_name -> milvus.proto.common.Status + 128, // 176: milvus.proto.query.DropIndexRequest.base:type_name -> milvus.proto.common.MsgBase + 144, // 177: milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry.value:type_name -> milvus.proto.rg.ResourceGroupConfig + 137, // 178: milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry.value:type_name -> milvus.proto.data.SegmentInfo + 147, // 179: milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats + 148, // 180: milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry.value:type_name -> milvus.proto.data.JsonKeyStats + 139, // 181: milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry.value:type_name -> milvus.proto.msg.MsgPosition + 54, // 182: milvus.proto.query.LeaderView.SegmentDistEntry.value:type_name -> milvus.proto.query.SegmentDist + 139, // 183: milvus.proto.query.LeaderView.GrowingSegmentsEntry.value:type_name -> milvus.proto.msg.MsgPosition + 29, // 184: milvus.proto.query.SegmentVersionInfo.IndexInfoEntry.value:type_name -> milvus.proto.query.FieldIndexInfo + 30, // 185: milvus.proto.query.SegmentVersionInfo.JsonStatsInfoEntry.value:type_name -> milvus.proto.query.JsonStatsInfo + 59, // 186: milvus.proto.query.Replica.ChannelNodeInfosEntry.value:type_name -> milvus.proto.query.ChannelNodeInfo + 7, // 187: milvus.proto.query.QueryCoord.ShowLoadCollections:input_type -> milvus.proto.query.ShowCollectionsRequest + 9, // 188: milvus.proto.query.QueryCoord.ShowLoadPartitions:input_type -> milvus.proto.query.ShowPartitionsRequest + 14, // 189: milvus.proto.query.QueryCoord.LoadPartitions:input_type -> milvus.proto.query.LoadPartitionsRequest + 15, // 190: milvus.proto.query.QueryCoord.ReleasePartitions:input_type -> milvus.proto.query.ReleasePartitionsRequest + 11, // 191: milvus.proto.query.QueryCoord.LoadCollection:input_type -> milvus.proto.query.LoadCollectionRequest + 12, // 192: milvus.proto.query.QueryCoord.ReleaseCollection:input_type -> milvus.proto.query.ReleaseCollectionRequest + 24, // 193: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:input_type -> milvus.proto.query.SyncNewCreatedPartitionRequest + 16, // 194: milvus.proto.query.QueryCoord.GetPartitionStates:input_type -> milvus.proto.query.GetPartitionStatesRequest + 18, // 195: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:input_type -> milvus.proto.query.GetSegmentInfoRequest + 40, // 196: milvus.proto.query.QueryCoord.LoadBalance:input_type -> milvus.proto.query.LoadBalanceRequest + 149, // 197: milvus.proto.query.QueryCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 150, // 198: milvus.proto.query.QueryCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 151, // 199: milvus.proto.query.QueryCoord.GetReplicas:input_type -> milvus.proto.milvus.GetReplicasRequest + 20, // 200: milvus.proto.query.QueryCoord.GetShardLeaders:input_type -> milvus.proto.query.GetShardLeadersRequest + 152, // 201: milvus.proto.query.QueryCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest + 153, // 202: milvus.proto.query.QueryCoord.CreateResourceGroup:input_type -> milvus.proto.milvus.CreateResourceGroupRequest + 22, // 203: milvus.proto.query.QueryCoord.UpdateResourceGroups:input_type -> milvus.proto.query.UpdateResourceGroupsRequest + 154, // 204: milvus.proto.query.QueryCoord.DropResourceGroup:input_type -> milvus.proto.milvus.DropResourceGroupRequest + 155, // 205: milvus.proto.query.QueryCoord.TransferNode:input_type -> milvus.proto.milvus.TransferNodeRequest + 64, // 206: milvus.proto.query.QueryCoord.TransferReplica:input_type -> milvus.proto.query.TransferReplicaRequest + 156, // 207: milvus.proto.query.QueryCoord.ListResourceGroups:input_type -> milvus.proto.milvus.ListResourceGroupsRequest + 65, // 208: milvus.proto.query.QueryCoord.DescribeResourceGroup:input_type -> milvus.proto.query.DescribeResourceGroupRequest + 104, // 209: milvus.proto.query.QueryCoord.ListLoadedSegments:input_type -> milvus.proto.query.ListLoadedSegmentsRequest + 73, // 210: milvus.proto.query.QueryCoord.ListCheckers:input_type -> milvus.proto.query.ListCheckersRequest + 71, // 211: milvus.proto.query.QueryCoord.ActivateChecker:input_type -> milvus.proto.query.ActivateCheckerRequest + 72, // 212: milvus.proto.query.QueryCoord.DeactivateChecker:input_type -> milvus.proto.query.DeactivateCheckerRequest + 81, // 213: milvus.proto.query.QueryCoord.ListQueryNode:input_type -> milvus.proto.query.ListQueryNodeRequest + 83, // 214: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:input_type -> milvus.proto.query.GetQueryNodeDistributionRequest + 85, // 215: milvus.proto.query.QueryCoord.SuspendBalance:input_type -> milvus.proto.query.SuspendBalanceRequest + 86, // 216: milvus.proto.query.QueryCoord.ResumeBalance:input_type -> milvus.proto.query.ResumeBalanceRequest + 87, // 217: milvus.proto.query.QueryCoord.CheckBalanceStatus:input_type -> milvus.proto.query.CheckBalanceStatusRequest + 89, // 218: milvus.proto.query.QueryCoord.SuspendNode:input_type -> milvus.proto.query.SuspendNodeRequest + 90, // 219: milvus.proto.query.QueryCoord.ResumeNode:input_type -> milvus.proto.query.ResumeNodeRequest + 91, // 220: milvus.proto.query.QueryCoord.TransferSegment:input_type -> milvus.proto.query.TransferSegmentRequest + 92, // 221: milvus.proto.query.QueryCoord.TransferChannel:input_type -> milvus.proto.query.TransferChannelRequest + 93, // 222: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:input_type -> milvus.proto.query.CheckQueryNodeDistributionRequest + 94, // 223: milvus.proto.query.QueryCoord.UpdateLoadConfig:input_type -> milvus.proto.query.UpdateLoadConfigRequest + 96, // 224: milvus.proto.query.QueryCoord.RunAnalyzer:input_type -> milvus.proto.query.RunAnalyzerRequest + 98, // 225: milvus.proto.query.QueryCoord.ValidateAnalyzer:input_type -> milvus.proto.query.ValidateAnalyzerRequest + 157, // 226: milvus.proto.query.QueryNode.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 158, // 227: milvus.proto.query.QueryNode.GetTimeTickChannel:input_type -> milvus.proto.internal.GetTimeTickChannelRequest + 159, // 228: milvus.proto.query.QueryNode.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 26, // 229: milvus.proto.query.QueryNode.WatchDmChannels:input_type -> milvus.proto.query.WatchDmChannelsRequest + 27, // 230: milvus.proto.query.QueryNode.UnsubDmChannel:input_type -> milvus.proto.query.UnsubDmChannelRequest + 31, // 231: milvus.proto.query.QueryNode.LoadSegments:input_type -> milvus.proto.query.LoadSegmentsRequest + 12, // 232: milvus.proto.query.QueryNode.ReleaseCollection:input_type -> milvus.proto.query.ReleaseCollectionRequest + 14, // 233: milvus.proto.query.QueryNode.LoadPartitions:input_type -> milvus.proto.query.LoadPartitionsRequest + 15, // 234: milvus.proto.query.QueryNode.ReleasePartitions:input_type -> milvus.proto.query.ReleasePartitionsRequest + 32, // 235: milvus.proto.query.QueryNode.ReleaseSegments:input_type -> milvus.proto.query.ReleaseSegmentsRequest + 18, // 236: milvus.proto.query.QueryNode.GetSegmentInfo:input_type -> milvus.proto.query.GetSegmentInfoRequest + 35, // 237: milvus.proto.query.QueryNode.SyncReplicaSegments:input_type -> milvus.proto.query.SyncReplicaSegmentsRequest + 13, // 238: milvus.proto.query.QueryNode.GetStatistics:input_type -> milvus.proto.query.GetStatisticsRequest + 33, // 239: milvus.proto.query.QueryNode.Search:input_type -> milvus.proto.query.SearchRequest + 33, // 240: milvus.proto.query.QueryNode.SearchSegments:input_type -> milvus.proto.query.SearchRequest + 34, // 241: milvus.proto.query.QueryNode.Query:input_type -> milvus.proto.query.QueryRequest + 34, // 242: milvus.proto.query.QueryNode.QueryStream:input_type -> milvus.proto.query.QueryRequest + 34, // 243: milvus.proto.query.QueryNode.QuerySegments:input_type -> milvus.proto.query.QueryRequest + 34, // 244: milvus.proto.query.QueryNode.QueryStreamSegments:input_type -> milvus.proto.query.QueryRequest + 149, // 245: milvus.proto.query.QueryNode.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 150, // 246: milvus.proto.query.QueryNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 50, // 247: milvus.proto.query.QueryNode.GetDataDistribution:input_type -> milvus.proto.query.GetDataDistributionRequest + 62, // 248: milvus.proto.query.QueryNode.SyncDistribution:input_type -> milvus.proto.query.SyncDistributionRequest + 68, // 249: milvus.proto.query.QueryNode.Delete:input_type -> milvus.proto.query.DeleteRequest + 69, // 250: milvus.proto.query.QueryNode.DeleteBatch:input_type -> milvus.proto.query.DeleteBatchRequest + 95, // 251: milvus.proto.query.QueryNode.UpdateSchema:input_type -> milvus.proto.query.UpdateSchemaRequest + 96, // 252: milvus.proto.query.QueryNode.RunAnalyzer:input_type -> milvus.proto.query.RunAnalyzerRequest + 100, // 253: milvus.proto.query.QueryNode.GetHighlight:input_type -> milvus.proto.query.GetHighlightRequest + 106, // 254: milvus.proto.query.QueryNode.DropIndex:input_type -> milvus.proto.query.DropIndexRequest + 98, // 255: milvus.proto.query.QueryNode.ValidateAnalyzer:input_type -> milvus.proto.query.ValidateAnalyzerRequest + 8, // 256: milvus.proto.query.QueryCoord.ShowLoadCollections:output_type -> milvus.proto.query.ShowCollectionsResponse + 10, // 257: milvus.proto.query.QueryCoord.ShowLoadPartitions:output_type -> milvus.proto.query.ShowPartitionsResponse + 129, // 258: milvus.proto.query.QueryCoord.LoadPartitions:output_type -> milvus.proto.common.Status + 129, // 259: milvus.proto.query.QueryCoord.ReleasePartitions:output_type -> milvus.proto.common.Status + 129, // 260: milvus.proto.query.QueryCoord.LoadCollection:output_type -> milvus.proto.common.Status + 129, // 261: milvus.proto.query.QueryCoord.ReleaseCollection:output_type -> milvus.proto.common.Status + 129, // 262: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:output_type -> milvus.proto.common.Status + 17, // 263: milvus.proto.query.QueryCoord.GetPartitionStates:output_type -> milvus.proto.query.GetPartitionStatesResponse + 19, // 264: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse + 129, // 265: milvus.proto.query.QueryCoord.LoadBalance:output_type -> milvus.proto.common.Status + 160, // 266: milvus.proto.query.QueryCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 161, // 267: milvus.proto.query.QueryCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 162, // 268: milvus.proto.query.QueryCoord.GetReplicas:output_type -> milvus.proto.milvus.GetReplicasResponse + 21, // 269: milvus.proto.query.QueryCoord.GetShardLeaders:output_type -> milvus.proto.query.GetShardLeadersResponse + 163, // 270: milvus.proto.query.QueryCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse + 129, // 271: milvus.proto.query.QueryCoord.CreateResourceGroup:output_type -> milvus.proto.common.Status + 129, // 272: milvus.proto.query.QueryCoord.UpdateResourceGroups:output_type -> milvus.proto.common.Status + 129, // 273: milvus.proto.query.QueryCoord.DropResourceGroup:output_type -> milvus.proto.common.Status + 129, // 274: milvus.proto.query.QueryCoord.TransferNode:output_type -> milvus.proto.common.Status + 129, // 275: milvus.proto.query.QueryCoord.TransferReplica:output_type -> milvus.proto.common.Status + 164, // 276: milvus.proto.query.QueryCoord.ListResourceGroups:output_type -> milvus.proto.milvus.ListResourceGroupsResponse + 66, // 277: milvus.proto.query.QueryCoord.DescribeResourceGroup:output_type -> milvus.proto.query.DescribeResourceGroupResponse + 105, // 278: milvus.proto.query.QueryCoord.ListLoadedSegments:output_type -> milvus.proto.query.ListLoadedSegmentsResponse + 74, // 279: milvus.proto.query.QueryCoord.ListCheckers:output_type -> milvus.proto.query.ListCheckersResponse + 129, // 280: milvus.proto.query.QueryCoord.ActivateChecker:output_type -> milvus.proto.common.Status + 129, // 281: milvus.proto.query.QueryCoord.DeactivateChecker:output_type -> milvus.proto.common.Status + 82, // 282: milvus.proto.query.QueryCoord.ListQueryNode:output_type -> milvus.proto.query.ListQueryNodeResponse + 84, // 283: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:output_type -> milvus.proto.query.GetQueryNodeDistributionResponse + 129, // 284: milvus.proto.query.QueryCoord.SuspendBalance:output_type -> milvus.proto.common.Status + 129, // 285: milvus.proto.query.QueryCoord.ResumeBalance:output_type -> milvus.proto.common.Status + 88, // 286: milvus.proto.query.QueryCoord.CheckBalanceStatus:output_type -> milvus.proto.query.CheckBalanceStatusResponse + 129, // 287: milvus.proto.query.QueryCoord.SuspendNode:output_type -> milvus.proto.common.Status + 129, // 288: milvus.proto.query.QueryCoord.ResumeNode:output_type -> milvus.proto.common.Status + 129, // 289: milvus.proto.query.QueryCoord.TransferSegment:output_type -> milvus.proto.common.Status + 129, // 290: milvus.proto.query.QueryCoord.TransferChannel:output_type -> milvus.proto.common.Status + 129, // 291: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:output_type -> milvus.proto.common.Status + 129, // 292: milvus.proto.query.QueryCoord.UpdateLoadConfig:output_type -> milvus.proto.common.Status + 165, // 293: milvus.proto.query.QueryCoord.RunAnalyzer:output_type -> milvus.proto.milvus.RunAnalyzerResponse + 129, // 294: milvus.proto.query.QueryCoord.ValidateAnalyzer:output_type -> milvus.proto.common.Status + 166, // 295: milvus.proto.query.QueryNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 167, // 296: milvus.proto.query.QueryNode.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse + 167, // 297: milvus.proto.query.QueryNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 129, // 298: milvus.proto.query.QueryNode.WatchDmChannels:output_type -> milvus.proto.common.Status + 129, // 299: milvus.proto.query.QueryNode.UnsubDmChannel:output_type -> milvus.proto.common.Status + 129, // 300: milvus.proto.query.QueryNode.LoadSegments:output_type -> milvus.proto.common.Status + 129, // 301: milvus.proto.query.QueryNode.ReleaseCollection:output_type -> milvus.proto.common.Status + 129, // 302: milvus.proto.query.QueryNode.LoadPartitions:output_type -> milvus.proto.common.Status + 129, // 303: milvus.proto.query.QueryNode.ReleasePartitions:output_type -> milvus.proto.common.Status + 129, // 304: milvus.proto.query.QueryNode.ReleaseSegments:output_type -> milvus.proto.common.Status + 19, // 305: milvus.proto.query.QueryNode.GetSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse + 129, // 306: milvus.proto.query.QueryNode.SyncReplicaSegments:output_type -> milvus.proto.common.Status + 168, // 307: milvus.proto.query.QueryNode.GetStatistics:output_type -> milvus.proto.internal.GetStatisticsResponse + 169, // 308: milvus.proto.query.QueryNode.Search:output_type -> milvus.proto.internal.SearchResults + 169, // 309: milvus.proto.query.QueryNode.SearchSegments:output_type -> milvus.proto.internal.SearchResults + 170, // 310: milvus.proto.query.QueryNode.Query:output_type -> milvus.proto.internal.RetrieveResults + 170, // 311: milvus.proto.query.QueryNode.QueryStream:output_type -> milvus.proto.internal.RetrieveResults + 170, // 312: milvus.proto.query.QueryNode.QuerySegments:output_type -> milvus.proto.internal.RetrieveResults + 170, // 313: milvus.proto.query.QueryNode.QueryStreamSegments:output_type -> milvus.proto.internal.RetrieveResults + 160, // 314: milvus.proto.query.QueryNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 161, // 315: milvus.proto.query.QueryNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 51, // 316: milvus.proto.query.QueryNode.GetDataDistribution:output_type -> milvus.proto.query.GetDataDistributionResponse + 129, // 317: milvus.proto.query.QueryNode.SyncDistribution:output_type -> milvus.proto.common.Status + 129, // 318: milvus.proto.query.QueryNode.Delete:output_type -> milvus.proto.common.Status + 70, // 319: milvus.proto.query.QueryNode.DeleteBatch:output_type -> milvus.proto.query.DeleteBatchResponse + 129, // 320: milvus.proto.query.QueryNode.UpdateSchema:output_type -> milvus.proto.common.Status + 165, // 321: milvus.proto.query.QueryNode.RunAnalyzer:output_type -> milvus.proto.milvus.RunAnalyzerResponse + 103, // 322: milvus.proto.query.QueryNode.GetHighlight:output_type -> milvus.proto.query.GetHighlightResponse + 129, // 323: milvus.proto.query.QueryNode.DropIndex:output_type -> milvus.proto.common.Status + 129, // 324: milvus.proto.query.QueryNode.ValidateAnalyzer:output_type -> milvus.proto.common.Status + 256, // [256:325] is the sub-list for method output_type + 187, // [187:256] is the sub-list for method input_type + 187, // [187:187] is the sub-list for extension type_name + 187, // [187:187] is the sub-list for extension extendee + 0, // [0:187] is the sub-list for field type_name } func init() { file_query_coord_proto_init() } @@ -11629,7 +12020,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[92].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListLoadedSegmentsRequest); i { + switch v := v.(*HighlightTask); i { case 0: return &v.state case 1: @@ -11641,7 +12032,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[93].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListLoadedSegmentsResponse); i { + switch v := v.(*GetHighlightRequest); i { case 0: return &v.state case 1: @@ -11653,6 +12044,66 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[94].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*HighlightFragment); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[95].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*HighlightResult); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[96].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetHighlightResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[97].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListLoadedSegmentsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[98].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListLoadedSegmentsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[99].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*DropIndexRequest); i { case 0: return &v.state @@ -11671,7 +12122,7 @@ func file_query_coord_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_query_coord_proto_rawDesc, NumEnums: 7, - NumMessages: 116, + NumMessages: 121, NumExtensions: 0, NumServices: 2, }, diff --git a/pkg/proto/querypb/query_coord_grpc.pb.go b/pkg/proto/querypb/query_coord_grpc.pb.go index 3badced8af..78fcb1cd11 100644 --- a/pkg/proto/querypb/query_coord_grpc.pb.go +++ b/pkg/proto/querypb/query_coord_grpc.pb.go @@ -1552,6 +1552,7 @@ const ( QueryNode_DeleteBatch_FullMethodName = "/milvus.proto.query.QueryNode/DeleteBatch" QueryNode_UpdateSchema_FullMethodName = "/milvus.proto.query.QueryNode/UpdateSchema" QueryNode_RunAnalyzer_FullMethodName = "/milvus.proto.query.QueryNode/RunAnalyzer" + QueryNode_GetHighlight_FullMethodName = "/milvus.proto.query.QueryNode/GetHighlight" QueryNode_DropIndex_FullMethodName = "/milvus.proto.query.QueryNode/DropIndex" QueryNode_ValidateAnalyzer_FullMethodName = "/milvus.proto.query.QueryNode/ValidateAnalyzer" ) @@ -1590,6 +1591,7 @@ type QueryNodeClient interface { DeleteBatch(ctx context.Context, in *DeleteBatchRequest, opts ...grpc.CallOption) (*DeleteBatchResponse, error) UpdateSchema(ctx context.Context, in *UpdateSchemaRequest, opts ...grpc.CallOption) (*commonpb.Status, error) RunAnalyzer(ctx context.Context, in *RunAnalyzerRequest, opts ...grpc.CallOption) (*milvuspb.RunAnalyzerResponse, error) + GetHighlight(ctx context.Context, in *GetHighlightRequest, opts ...grpc.CallOption) (*GetHighlightResponse, error) DropIndex(ctx context.Context, in *DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) ValidateAnalyzer(ctx context.Context, in *ValidateAnalyzerRequest, opts ...grpc.CallOption) (*commonpb.Status, error) } @@ -1891,6 +1893,15 @@ func (c *queryNodeClient) RunAnalyzer(ctx context.Context, in *RunAnalyzerReques return out, nil } +func (c *queryNodeClient) GetHighlight(ctx context.Context, in *GetHighlightRequest, opts ...grpc.CallOption) (*GetHighlightResponse, error) { + out := new(GetHighlightResponse) + err := c.cc.Invoke(ctx, QueryNode_GetHighlight_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *queryNodeClient) DropIndex(ctx context.Context, in *DropIndexRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { out := new(commonpb.Status) err := c.cc.Invoke(ctx, QueryNode_DropIndex_FullMethodName, in, out, opts...) @@ -1943,6 +1954,7 @@ type QueryNodeServer interface { DeleteBatch(context.Context, *DeleteBatchRequest) (*DeleteBatchResponse, error) UpdateSchema(context.Context, *UpdateSchemaRequest) (*commonpb.Status, error) RunAnalyzer(context.Context, *RunAnalyzerRequest) (*milvuspb.RunAnalyzerResponse, error) + GetHighlight(context.Context, *GetHighlightRequest) (*GetHighlightResponse, error) DropIndex(context.Context, *DropIndexRequest) (*commonpb.Status, error) ValidateAnalyzer(context.Context, *ValidateAnalyzerRequest) (*commonpb.Status, error) } @@ -2032,6 +2044,9 @@ func (UnimplementedQueryNodeServer) UpdateSchema(context.Context, *UpdateSchemaR func (UnimplementedQueryNodeServer) RunAnalyzer(context.Context, *RunAnalyzerRequest) (*milvuspb.RunAnalyzerResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method RunAnalyzer not implemented") } +func (UnimplementedQueryNodeServer) GetHighlight(context.Context, *GetHighlightRequest) (*GetHighlightResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method GetHighlight not implemented") +} func (UnimplementedQueryNodeServer) DropIndex(context.Context, *DropIndexRequest) (*commonpb.Status, error) { return nil, status.Errorf(codes.Unimplemented, "method DropIndex not implemented") } @@ -2542,6 +2557,24 @@ func _QueryNode_RunAnalyzer_Handler(srv interface{}, ctx context.Context, dec fu return interceptor(ctx, in, info, handler) } +func _QueryNode_GetHighlight_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(GetHighlightRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).GetHighlight(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_GetHighlight_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).GetHighlight(ctx, req.(*GetHighlightRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _QueryNode_DropIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(DropIndexRequest) if err := dec(in); err != nil { @@ -2685,6 +2718,10 @@ var QueryNode_ServiceDesc = grpc.ServiceDesc{ MethodName: "RunAnalyzer", Handler: _QueryNode_RunAnalyzer_Handler, }, + { + MethodName: "GetHighlight", + Handler: _QueryNode_GetHighlight_Handler, + }, { MethodName: "DropIndex", Handler: _QueryNode_DropIndex_Handler,