From b36c88f3c824d4a5ae2d20893ae351bba595c685 Mon Sep 17 00:00:00 2001 From: congqixia Date: Tue, 22 Apr 2025 16:28:37 +0800 Subject: [PATCH] enhance: [AddField] Broadcast schema change via WAL (#41373) Related to #39718 Add Broadcast logic for collection schema change and notifies: - Streamnode - Delegator - Streamnode - Flush component - QueryNodes via grpc --------- Signed-off-by: Congqi Xia --- internal/core/src/segcore/Collection.cpp | 12 + internal/core/src/segcore/Collection.h | 3 + internal/core/src/segcore/collection_c.cpp | 16 + internal/core/src/segcore/collection_c.h | 5 + internal/core/unittest/test_c_api.cpp | 68 +- internal/core/unittest/test_utils/DataGen.h | 1 - .../msghandlerimpl/msg_handler_impl.go | 8 + .../distributed/querynode/client/client.go | 11 + internal/distributed/querynode/service.go | 4 + internal/flushcommon/metacache/meta_cache.go | 6 + .../flushcommon/metacache/mock_meta_cache.go | 33 + .../pipeline/flow_graph_dd_node.go | 15 + .../pipeline/flow_graph_message.go | 3 + .../pipeline/flow_graph_write_node.go | 5 + internal/flushcommon/util/msg_handler.go | 3 + .../flushcommon/mock_util/mock_MsgHandler.go | 50 + internal/mocks/mock_querynode.go | 59 + internal/mocks/mock_querynode_client.go | 74 + internal/querycoordv2/mocks/mock_querynode.go | 59 + internal/querynodev2/cluster/mock_worker.go | 62 + internal/querynodev2/cluster/worker.go | 7 + internal/querynodev2/delegator/delegator.go | 47 + .../querynodev2/delegator/delegator_test.go | 93 + .../querynodev2/delegator/mock_delegator.go | 49 + internal/querynodev2/local_worker.go | 5 + internal/querynodev2/pipeline/delete_node.go | 5 + internal/querynodev2/pipeline/filter_node.go | 8 + internal/querynodev2/pipeline/insert_node.go | 1 + internal/querynodev2/pipeline/message.go | 11 + internal/querynodev2/segments/collection.go | 19 + .../querynodev2/segments/collection_test.go | 93 + .../segments/mock_collection_manager.go | 47 + internal/querynodev2/services.go | 22 + internal/querynodev2/services_test.go | 38 + internal/rootcoord/add_field_task.go | 11 +- internal/rootcoord/add_field_task_test.go | 162 +- internal/rootcoord/step.go | 73 + .../flusher/flusherimpl/msg_handler_impl.go | 17 +- .../server/wal/{RAEDME.md => README.md} | 0 .../server/wal/adaptor/initializing.go | 2 +- .../timetick/timetick_sync_operator_test.go | 2 +- .../interceptors/wab/write_ahead_buffer.go | 2 +- .../wab/write_ahead_buffer_test.go | 6 +- internal/util/mock/grpc_querynode_client.go | 4 + internal/util/pipeline/stream_pipeline.go | 3 +- internal/util/segcore/collection.go | 15 + internal/util/wrappers/qn_wrapper.go | 4 + pkg/proto/messages.proto | 14 + pkg/proto/messagespb/messages.pb.go | 660 ++++--- pkg/proto/query_coord.proto | 7 + pkg/proto/querypb/query_coord.pb.go | 1525 +++++++++-------- pkg/proto/querypb/query_coord_grpc.pb.go | 37 + pkg/streaming/util/message/adaptor/message.go | 2 + .../util/message/adaptor/message_type.go | 1 + .../util/message/adaptor/ts_msg_newer.go | 26 + pkg/streaming/util/message/broadcast.go | 8 +- pkg/streaming/util/message/builder.go | 1 + pkg/streaming/util/message/message_type.go | 2 + .../util/message/specialized_message.go | 6 + 59 files changed, 2488 insertions(+), 1044 deletions(-) create mode 100644 internal/querynodev2/segments/collection_test.go rename internal/streamingnode/server/wal/{RAEDME.md => README.md} (100%) diff --git a/internal/core/src/segcore/Collection.cpp b/internal/core/src/segcore/Collection.cpp index 6cd74255e5..aa2871460b 100644 --- a/internal/core/src/segcore/Collection.cpp +++ b/internal/core/src/segcore/Collection.cpp @@ -64,4 +64,16 @@ Collection::parseIndexMeta(const void* index_proto, const int64_t length) { LOG_INFO("index meta info: {}", index_meta_->ToString()); } +void +Collection::parse_schema(const void* schema_proto_blob, const int64_t length) { + Assert(schema_proto_blob != nullptr); + + milvus::proto::schema::CollectionSchema collection_schema; + auto suc = collection_schema.ParseFromArray(schema_proto_blob, length); + + AssertInfo(suc, "parse schema proto failed"); + + schema_ = Schema::ParseFrom(collection_schema); +} + } // namespace milvus::segcore diff --git a/internal/core/src/segcore/Collection.h b/internal/core/src/segcore/Collection.h index 0c8f64abfb..cd5c377b61 100644 --- a/internal/core/src/segcore/Collection.h +++ b/internal/core/src/segcore/Collection.h @@ -28,6 +28,9 @@ class Collection { void parseIndexMeta(const void* index_meta_proto_blob, const int64_t length); + void + parse_schema(const void* schema_proto_blob, const int64_t length); + public: SchemaPtr& get_schema() { diff --git a/internal/core/src/segcore/collection_c.cpp b/internal/core/src/segcore/collection_c.cpp index 39ebf29ba7..7775a9ac3c 100644 --- a/internal/core/src/segcore/collection_c.cpp +++ b/internal/core/src/segcore/collection_c.cpp @@ -9,6 +9,8 @@ // is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express // or implied. See the License for the specific language governing permissions and limitations under the License +#include +#include "common/EasyAssert.h" #include "common/type_c.h" #ifdef __linux__ #include @@ -32,6 +34,20 @@ NewCollection(const void* schema_proto_blob, } } +CStatus +UpdateSchema(CCollection collection, + const void* proto_blob, + const int64_t length) { + try { + auto col = static_cast(collection); + + col->parse_schema(proto_blob, length); + return milvus::SuccessCStatus(); + } catch (std::exception& e) { + return milvus::FailureCStatus(&e); + } +} + CStatus SetIndexMeta(CCollection collection, const void* proto_blob, diff --git a/internal/core/src/segcore/collection_c.h b/internal/core/src/segcore/collection_c.h index 3cd20df10f..74cd8c8004 100644 --- a/internal/core/src/segcore/collection_c.h +++ b/internal/core/src/segcore/collection_c.h @@ -25,6 +25,11 @@ NewCollection(const void* schema_proto_blob, const int64_t length, CCollection* collection); +CStatus +UpdateSchema(CCollection collection, + const void* proto_blob, + const int64_t length); + CStatus SetIndexMeta(CCollection collection, const void* proto_blob, diff --git a/internal/core/unittest/test_c_api.cpp b/internal/core/unittest/test_c_api.cpp index c8be25bbbf..280d00b557 100644 --- a/internal/core/unittest/test_c_api.cpp +++ b/internal/core/unittest/test_c_api.cpp @@ -30,7 +30,9 @@ #include "index/IndexFactory.h" #include "knowhere/comp/index_param.h" #include "pb/plan.pb.h" +#include "pb/schema.pb.h" #include "segcore/Collection.h" +#include "segcore/collection_c.h" #include "segcore/reduce/Reduce.h" #include "segcore/reduce_c.h" #include "segcore/segment_c.h" @@ -239,6 +241,71 @@ TEST(CApiTest, CollectionTest) { DeleteCollection(collection); } +TEST(CApiTest, UpdateSchemaTest) { + std::string schema_string = generate_collection_schema( + knowhere::metric::L2, DIM); + // CCollection + auto collection = NewCollection(schema_string.c_str()); + + // create updated schema with extra field + namespace schema = milvus::proto::schema; + schema::CollectionSchema collection_schema; + collection_schema.set_name("collection_test"); + + auto vec_field_schema = collection_schema.add_fields(); + vec_field_schema->set_name("fakevec"); + vec_field_schema->set_fieldid(100); + vec_field_schema->set_data_type(schema::DataType::FloatVector); + auto metric_type_param = vec_field_schema->add_index_params(); + metric_type_param->set_key("metric_type"); + metric_type_param->set_value(knowhere::metric::L2); + auto dim_param = vec_field_schema->add_type_params(); + dim_param->set_key("dim"); + dim_param->set_value(std::to_string(DIM)); + + auto other_field_schema = collection_schema.add_fields(); + other_field_schema->set_name("counter"); + other_field_schema->set_fieldid(101); + other_field_schema->set_data_type(schema::DataType::Int64); + other_field_schema->set_is_primary_key(true); + + auto other_field_schema2 = collection_schema.add_fields(); + other_field_schema2->set_name("doubleField"); + other_field_schema2->set_fieldid(102); + other_field_schema2->set_data_type(schema::DataType::Double); + + auto added_field = collection_schema.add_fields(); + added_field->set_name("added_field"); + added_field->set_fieldid(103); + added_field->set_data_type(schema::DataType::Int64); + added_field->set_nullable(true); + + std::string updated_schema_string; + auto marshal = collection_schema.SerializeToString(&updated_schema_string); + ASSERT_TRUE(marshal); + + // Call UpdateSchema CApi here + // UpdateSchema(CCollection, const void*, const int64_t) + auto status = UpdateSchema(collection, + updated_schema_string.c_str(), + updated_schema_string.length()); + ASSERT_EQ(status.error_code, Success); + + auto col = static_cast(collection); + auto updated_schema = col->get_schema(); + auto add_field = updated_schema->operator[](FieldId(103)); + + ASSERT_EQ(add_field.get_name().get(), "added_field"); + + // Test failure case, no panicking with failure code + status = UpdateSchema(collection, nullptr, 0); + ASSERT_NE(status.error_code, Success); + + DeleteCollection(collection); + // free error msg, which shall be responsible for go side to call C.free() + free(const_cast(status.error_msg)); +} + TEST(CApiTest, LoadInfoTest) { auto load_info = std::make_shared(); auto c_load_info = reinterpret_cast(load_info.get()); @@ -4231,7 +4298,6 @@ TEST(CApiTest, GrowingSegment_Load_Field_Data_Lack_Binlog_Rows) { DeleteSegment(segment); } - TEST(CApiTest, DISABLED_SealedSegment_Load_Field_Data_Lack_Binlog_Rows) { double double_default_value = 20; auto schema = std::make_shared(); diff --git a/internal/core/unittest/test_utils/DataGen.h b/internal/core/unittest/test_utils/DataGen.h index af899049bd..4417592e29 100644 --- a/internal/core/unittest/test_utils/DataGen.h +++ b/internal/core/unittest/test_utils/DataGen.h @@ -45,7 +45,6 @@ #include "storage/Util.h" #include "milvus-storage/common/constants.h" - using boost::algorithm::starts_with; namespace milvus::segcore { diff --git a/internal/datanode/msghandlerimpl/msg_handler_impl.go b/internal/datanode/msghandlerimpl/msg_handler_impl.go index db7f8ba6ac..c53b36ab27 100644 --- a/internal/datanode/msghandlerimpl/msg_handler_impl.go +++ b/internal/datanode/msghandlerimpl/msg_handler_impl.go @@ -32,6 +32,7 @@ import ( "github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "github.com/milvus-io/milvus/pkg/v2/streaming/util/message" + "github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor" "github.com/milvus-io/milvus/pkg/v2/streaming/util/types" "github.com/milvus-io/milvus/pkg/v2/util/funcutil" "github.com/milvus-io/milvus/pkg/v2/util/merr" @@ -89,6 +90,13 @@ func (m *msgHandlerImpl) HandleImport(ctx context.Context, vchannel string, impo }, retry.AttemptAlways()) } +func (impl *msgHandlerImpl) HandleSchemaChange(ctx context.Context, vchannel string, msg *adaptor.SchemaChangeMessageBody) error { + return streaming.WAL().Broadcast().Ack(ctx, types.BroadcastAckRequest{ + BroadcastID: msg.BroadcastID, + VChannel: vchannel, + }) +} + func NewMsgHandlerImpl(broker broker.Broker) *msgHandlerImpl { return &msgHandlerImpl{ broker: broker, diff --git a/internal/distributed/querynode/client/client.go b/internal/distributed/querynode/client/client.go index 0f6b4f55ae..3626fe7e91 100644 --- a/internal/distributed/querynode/client/client.go +++ b/internal/distributed/querynode/client/client.go @@ -360,3 +360,14 @@ func (c *Client) DeleteBatch(ctx context.Context, req *querypb.DeleteBatchReques return client.DeleteBatch(ctx, req) }) } + +func (c *Client) UpdateSchema(ctx context.Context, req *querypb.UpdateSchemaRequest, _ ...grpc.CallOption) (*commonpb.Status, error) { + req = typeutil.Clone(req) + commonpbutil.UpdateMsgBase( + req.GetBase(), + commonpbutil.FillMsgBaseFromClient(c.nodeID), + ) + return wrapGrpcCall(ctx, c, func(client querypb.QueryNodeClient) (*commonpb.Status, error) { + return client.UpdateSchema(ctx, req) + }) +} diff --git a/internal/distributed/querynode/service.go b/internal/distributed/querynode/service.go index a42b393943..47dd963b8d 100644 --- a/internal/distributed/querynode/service.go +++ b/internal/distributed/querynode/service.go @@ -394,3 +394,7 @@ func (s *Server) Delete(ctx context.Context, req *querypb.DeleteRequest) (*commo func (s *Server) DeleteBatch(ctx context.Context, req *querypb.DeleteBatchRequest) (*querypb.DeleteBatchResponse, error) { return s.querynode.DeleteBatch(ctx, req) } + +func (s *Server) UpdateSchema(ctx context.Context, req *querypb.UpdateSchemaRequest) (*commonpb.Status, error) { + return s.querynode.UpdateSchema(ctx, req) +} diff --git a/internal/flushcommon/metacache/meta_cache.go b/internal/flushcommon/metacache/meta_cache.go index 29638d61a4..a0811db59d 100644 --- a/internal/flushcommon/metacache/meta_cache.go +++ b/internal/flushcommon/metacache/meta_cache.go @@ -54,6 +54,8 @@ type MetaCache interface { DetectMissingSegments(segments map[int64]struct{}) []int64 // UpdateSegmentView updates the segments BF from datacoord view. UpdateSegmentView(partitionID int64, newSegments []*datapb.SyncSegmentInfo, newSegmentsBF []*pkoracle.BloomFilterSet, allSegments map[int64]struct{}) + // UpdateSchema update the latest collection schema + UpdateSchema(updatedSchema *schemapb.CollectionSchema) } var _ MetaCache = (*metaCacheImpl)(nil) @@ -316,3 +318,7 @@ func (c *metaCacheImpl) UpdateSegmentView(partitionID int64, } } } + +func (c *metaCacheImpl) UpdateSchema(updatedSchema *schemapb.CollectionSchema) { + c.schema = updatedSchema +} diff --git a/internal/flushcommon/metacache/mock_meta_cache.go b/internal/flushcommon/metacache/mock_meta_cache.go index 0acbfd0a6b..e7305be1ac 100644 --- a/internal/flushcommon/metacache/mock_meta_cache.go +++ b/internal/flushcommon/metacache/mock_meta_cache.go @@ -545,6 +545,39 @@ func (_c *MockMetaCache_Schema_Call) RunAndReturn(run func() *schemapb.Collectio return _c } +// UpdateSchema provides a mock function with given fields: updatedSchema +func (_m *MockMetaCache) UpdateSchema(updatedSchema *schemapb.CollectionSchema) { + _m.Called(updatedSchema) +} + +// MockMetaCache_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema' +type MockMetaCache_UpdateSchema_Call struct { + *mock.Call +} + +// UpdateSchema is a helper method to define mock.On call +// - updatedSchema *schemapb.CollectionSchema +func (_e *MockMetaCache_Expecter) UpdateSchema(updatedSchema interface{}) *MockMetaCache_UpdateSchema_Call { + return &MockMetaCache_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema", updatedSchema)} +} + +func (_c *MockMetaCache_UpdateSchema_Call) Run(run func(updatedSchema *schemapb.CollectionSchema)) *MockMetaCache_UpdateSchema_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(*schemapb.CollectionSchema)) + }) + return _c +} + +func (_c *MockMetaCache_UpdateSchema_Call) Return() *MockMetaCache_UpdateSchema_Call { + _c.Call.Return() + return _c +} + +func (_c *MockMetaCache_UpdateSchema_Call) RunAndReturn(run func(*schemapb.CollectionSchema)) *MockMetaCache_UpdateSchema_Call { + _c.Call.Return(run) + return _c +} + // UpdateSegmentView provides a mock function with given fields: partitionID, newSegments, newSegmentsBF, allSegments func (_m *MockMetaCache) UpdateSegmentView(partitionID int64, newSegments []*datapb.SyncSegmentInfo, newSegmentsBF []*pkoracle.BloomFilterSet, allSegments map[int64]struct{}) { _m.Called(partitionID, newSegments, newSegmentsBF, allSegments) diff --git a/internal/flushcommon/pipeline/flow_graph_dd_node.go b/internal/flushcommon/pipeline/flow_graph_dd_node.go index ae1e11d9ad..8b5b9cd3ac 100644 --- a/internal/flushcommon/pipeline/flow_graph_dd_node.go +++ b/internal/flushcommon/pipeline/flow_graph_dd_node.go @@ -296,6 +296,21 @@ func (ddn *ddNode) Operate(in []Msg) []Msg { } else { logger.Info("handle import message success") } + case commonpb.MsgType_AddCollectionField: + schemaMsg := msg.(*adaptor.SchemaChangeMessageBody) + header := schemaMsg.SchemaChangeMessage.Header() + if header.GetCollectionId() != ddn.collectionID { + continue + } + logger := log.With(zap.String("vchannel", ddn.Name())) + logger.Info("receive schema change message") + body, err := schemaMsg.SchemaChangeMessage.Body() + if err != nil { + logger.Warn("failed to unmarshal schema change message body", zap.Error(err)) + continue + } + fgMsg.updatedSchema = body.GetSchema() + ddn.msgHandler.HandleSchemaChange(ddn.ctx, ddn.vChannelName, schemaMsg) } } diff --git a/internal/flushcommon/pipeline/flow_graph_message.go b/internal/flushcommon/pipeline/flow_graph_message.go index cabe788a31..e4744094c0 100644 --- a/internal/flushcommon/pipeline/flow_graph_message.go +++ b/internal/flushcommon/pipeline/flow_graph_message.go @@ -18,6 +18,7 @@ package pipeline import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" "github.com/milvus-io/milvus/internal/storage" @@ -60,6 +61,8 @@ type FlowGraphMsg struct { segmentsToSync []typeutil.UniqueID dropCollection bool dropPartitions []typeutil.UniqueID + + updatedSchema *schemapb.CollectionSchema } func (fgMsg *FlowGraphMsg) TimeTick() typeutil.Timestamp { diff --git a/internal/flushcommon/pipeline/flow_graph_write_node.go b/internal/flushcommon/pipeline/flow_graph_write_node.go index 3ace91d5c4..518738a7f2 100644 --- a/internal/flushcommon/pipeline/flow_graph_write_node.go +++ b/internal/flushcommon/pipeline/flow_graph_write_node.go @@ -115,6 +115,11 @@ func (wNode *writeNode) Operate(in []Msg) []Msg { wNode.updater.Update(wNode.channelName, end.GetTimestamp(), stats) } + // update schema after all data processed + if fgMsg.updatedSchema != nil { + wNode.metacache.UpdateSchema(fgMsg.updatedSchema) + } + res := FlowGraphMsg{ TimeRange: fgMsg.TimeRange, StartPositions: fgMsg.StartPositions, diff --git a/internal/flushcommon/util/msg_handler.go b/internal/flushcommon/util/msg_handler.go index 4b2991e311..dfc250cec2 100644 --- a/internal/flushcommon/util/msg_handler.go +++ b/internal/flushcommon/util/msg_handler.go @@ -22,6 +22,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "github.com/milvus-io/milvus/pkg/v2/streaming/util/message" + "github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor" ) type MsgHandler interface { @@ -32,6 +33,8 @@ type MsgHandler interface { HandleManualFlush(vchannel string, flushMsg message.ImmutableManualFlushMessageV2) error HandleImport(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg) error + + HandleSchemaChange(ctx context.Context, vchannel string, msg *adaptor.SchemaChangeMessageBody) error } func ConvertInternalImportFile(file *msgpb.ImportFile, _ int) *internalpb.ImportFile { diff --git a/internal/mocks/flushcommon/mock_util/mock_MsgHandler.go b/internal/mocks/flushcommon/mock_util/mock_MsgHandler.go index ef2cc09f02..b6e41f2cdc 100644 --- a/internal/mocks/flushcommon/mock_util/mock_MsgHandler.go +++ b/internal/mocks/flushcommon/mock_util/mock_MsgHandler.go @@ -5,6 +5,8 @@ package mock_util import ( context "context" + adaptor "github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor" + message "github.com/milvus-io/milvus/pkg/v2/streaming/util/message" mock "github.com/stretchr/testify/mock" @@ -215,6 +217,54 @@ func (_c *MockMsgHandler_HandleManualFlush_Call) RunAndReturn(run func(string, m return _c } +// HandleSchemaChange provides a mock function with given fields: ctx, vchannel, msg +func (_m *MockMsgHandler) HandleSchemaChange(ctx context.Context, vchannel string, msg *adaptor.SchemaChangeMessageBody) error { + ret := _m.Called(ctx, vchannel, msg) + + if len(ret) == 0 { + panic("no return value specified for HandleSchemaChange") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, string, *adaptor.SchemaChangeMessageBody) error); ok { + r0 = rf(ctx, vchannel, msg) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockMsgHandler_HandleSchemaChange_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleSchemaChange' +type MockMsgHandler_HandleSchemaChange_Call struct { + *mock.Call +} + +// HandleSchemaChange is a helper method to define mock.On call +// - ctx context.Context +// - vchannel string +// - msg *adaptor.SchemaChangeMessageBody +func (_e *MockMsgHandler_Expecter) HandleSchemaChange(ctx interface{}, vchannel interface{}, msg interface{}) *MockMsgHandler_HandleSchemaChange_Call { + return &MockMsgHandler_HandleSchemaChange_Call{Call: _e.mock.On("HandleSchemaChange", ctx, vchannel, msg)} +} + +func (_c *MockMsgHandler_HandleSchemaChange_Call) Run(run func(ctx context.Context, vchannel string, msg *adaptor.SchemaChangeMessageBody)) *MockMsgHandler_HandleSchemaChange_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string), args[2].(*adaptor.SchemaChangeMessageBody)) + }) + return _c +} + +func (_c *MockMsgHandler_HandleSchemaChange_Call) Return(_a0 error) *MockMsgHandler_HandleSchemaChange_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockMsgHandler_HandleSchemaChange_Call) RunAndReturn(run func(context.Context, string, *adaptor.SchemaChangeMessageBody) error) *MockMsgHandler_HandleSchemaChange_Call { + _c.Call.Return(run) + return _c +} + // NewMockMsgHandler creates a new instance of MockMsgHandler. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. // The first argument is typically a *testing.T value. func NewMockMsgHandler(t interface { diff --git a/internal/mocks/mock_querynode.go b/internal/mocks/mock_querynode.go index aead5a45d6..f293f6ad6e 100644 --- a/internal/mocks/mock_querynode.go +++ b/internal/mocks/mock_querynode.go @@ -1758,6 +1758,65 @@ func (_c *MockQueryNode_UnsubDmChannel_Call) RunAndReturn(run func(context.Conte return _c } +// UpdateSchema provides a mock function with given fields: _a0, _a1 +func (_m *MockQueryNode) UpdateSchema(_a0 context.Context, _a1 *querypb.UpdateSchemaRequest) (*commonpb.Status, error) { + ret := _m.Called(_a0, _a1) + + if len(ret) == 0 { + panic("no return value specified for UpdateSchema") + } + + var r0 *commonpb.Status + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest) (*commonpb.Status, error)); ok { + return rf(_a0, _a1) + } + if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest) *commonpb.Status); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*commonpb.Status) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *querypb.UpdateSchemaRequest) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockQueryNode_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema' +type MockQueryNode_UpdateSchema_Call struct { + *mock.Call +} + +// UpdateSchema is a helper method to define mock.On call +// - _a0 context.Context +// - _a1 *querypb.UpdateSchemaRequest +func (_e *MockQueryNode_Expecter) UpdateSchema(_a0 interface{}, _a1 interface{}) *MockQueryNode_UpdateSchema_Call { + return &MockQueryNode_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema", _a0, _a1)} +} + +func (_c *MockQueryNode_UpdateSchema_Call) Run(run func(_a0 context.Context, _a1 *querypb.UpdateSchemaRequest)) *MockQueryNode_UpdateSchema_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*querypb.UpdateSchemaRequest)) + }) + return _c +} + +func (_c *MockQueryNode_UpdateSchema_Call) Return(_a0 *commonpb.Status, _a1 error) *MockQueryNode_UpdateSchema_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockQueryNode_UpdateSchema_Call) RunAndReturn(run func(context.Context, *querypb.UpdateSchemaRequest) (*commonpb.Status, error)) *MockQueryNode_UpdateSchema_Call { + _c.Call.Return(run) + return _c +} + // UpdateStateCode provides a mock function with given fields: stateCode func (_m *MockQueryNode) UpdateStateCode(stateCode commonpb.StateCode) { _m.Called(stateCode) diff --git a/internal/mocks/mock_querynode_client.go b/internal/mocks/mock_querynode_client.go index 9924fda6fe..9d6ca80c62 100644 --- a/internal/mocks/mock_querynode_client.go +++ b/internal/mocks/mock_querynode_client.go @@ -1852,6 +1852,80 @@ func (_c *MockQueryNodeClient_UnsubDmChannel_Call) RunAndReturn(run func(context return _c } +// UpdateSchema provides a mock function with given fields: ctx, in, opts +func (_m *MockQueryNodeClient) UpdateSchema(ctx context.Context, in *querypb.UpdateSchemaRequest, opts ...grpc.CallOption) (*commonpb.Status, 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 UpdateSchema") + } + + var r0 *commonpb.Status + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok { + return rf(ctx, in, opts...) + } + if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest, ...grpc.CallOption) *commonpb.Status); ok { + r0 = rf(ctx, in, opts...) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*commonpb.Status) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *querypb.UpdateSchemaRequest, ...grpc.CallOption) error); ok { + r1 = rf(ctx, in, opts...) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockQueryNodeClient_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema' +type MockQueryNodeClient_UpdateSchema_Call struct { + *mock.Call +} + +// UpdateSchema is a helper method to define mock.On call +// - ctx context.Context +// - in *querypb.UpdateSchemaRequest +// - opts ...grpc.CallOption +func (_e *MockQueryNodeClient_Expecter) UpdateSchema(ctx interface{}, in interface{}, opts ...interface{}) *MockQueryNodeClient_UpdateSchema_Call { + return &MockQueryNodeClient_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema", + append([]interface{}{ctx, in}, opts...)...)} +} + +func (_c *MockQueryNodeClient_UpdateSchema_Call) Run(run func(ctx context.Context, in *querypb.UpdateSchemaRequest, opts ...grpc.CallOption)) *MockQueryNodeClient_UpdateSchema_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.UpdateSchemaRequest), variadicArgs...) + }) + return _c +} + +func (_c *MockQueryNodeClient_UpdateSchema_Call) Return(_a0 *commonpb.Status, _a1 error) *MockQueryNodeClient_UpdateSchema_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockQueryNodeClient_UpdateSchema_Call) RunAndReturn(run func(context.Context, *querypb.UpdateSchemaRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockQueryNodeClient_UpdateSchema_Call { + _c.Call.Return(run) + return _c +} + // WatchDmChannels provides a mock function with given fields: ctx, in, opts func (_m *MockQueryNodeClient) WatchDmChannels(ctx context.Context, in *querypb.WatchDmChannelsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { _va := make([]interface{}, len(opts)) diff --git a/internal/querycoordv2/mocks/mock_querynode.go b/internal/querycoordv2/mocks/mock_querynode.go index 2905ea7924..43e482cc43 100644 --- a/internal/querycoordv2/mocks/mock_querynode.go +++ b/internal/querycoordv2/mocks/mock_querynode.go @@ -1421,6 +1421,65 @@ func (_c *MockQueryNodeServer_UnsubDmChannel_Call) RunAndReturn(run func(context return _c } +// UpdateSchema provides a mock function with given fields: _a0, _a1 +func (_m *MockQueryNodeServer) UpdateSchema(_a0 context.Context, _a1 *querypb.UpdateSchemaRequest) (*commonpb.Status, error) { + ret := _m.Called(_a0, _a1) + + if len(ret) == 0 { + panic("no return value specified for UpdateSchema") + } + + var r0 *commonpb.Status + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest) (*commonpb.Status, error)); ok { + return rf(_a0, _a1) + } + if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest) *commonpb.Status); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*commonpb.Status) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *querypb.UpdateSchemaRequest) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockQueryNodeServer_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema' +type MockQueryNodeServer_UpdateSchema_Call struct { + *mock.Call +} + +// UpdateSchema is a helper method to define mock.On call +// - _a0 context.Context +// - _a1 *querypb.UpdateSchemaRequest +func (_e *MockQueryNodeServer_Expecter) UpdateSchema(_a0 interface{}, _a1 interface{}) *MockQueryNodeServer_UpdateSchema_Call { + return &MockQueryNodeServer_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema", _a0, _a1)} +} + +func (_c *MockQueryNodeServer_UpdateSchema_Call) Run(run func(_a0 context.Context, _a1 *querypb.UpdateSchemaRequest)) *MockQueryNodeServer_UpdateSchema_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*querypb.UpdateSchemaRequest)) + }) + return _c +} + +func (_c *MockQueryNodeServer_UpdateSchema_Call) Return(_a0 *commonpb.Status, _a1 error) *MockQueryNodeServer_UpdateSchema_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockQueryNodeServer_UpdateSchema_Call) RunAndReturn(run func(context.Context, *querypb.UpdateSchemaRequest) (*commonpb.Status, error)) *MockQueryNodeServer_UpdateSchema_Call { + _c.Call.Return(run) + return _c +} + // WatchDmChannels provides a mock function with given fields: _a0, _a1 func (_m *MockQueryNodeServer) WatchDmChannels(_a0 context.Context, _a1 *querypb.WatchDmChannelsRequest) (*commonpb.Status, error) { ret := _m.Called(_a0, _a1) diff --git a/internal/querynodev2/cluster/mock_worker.go b/internal/querynodev2/cluster/mock_worker.go index f710f517aa..a4aaa5d68d 100644 --- a/internal/querynodev2/cluster/mock_worker.go +++ b/internal/querynodev2/cluster/mock_worker.go @@ -5,7 +5,10 @@ package cluster import ( context "context" + commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + internalpb "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" + mock "github.com/stretchr/testify/mock" querypb "github.com/milvus-io/milvus/pkg/v2/proto/querypb" @@ -528,6 +531,65 @@ func (_c *MockWorker_Stop_Call) RunAndReturn(run func()) *MockWorker_Stop_Call { return _c } +// UpdateSchema provides a mock function with given fields: ctx, req +func (_m *MockWorker) UpdateSchema(ctx context.Context, req *querypb.UpdateSchemaRequest) (*commonpb.Status, error) { + ret := _m.Called(ctx, req) + + if len(ret) == 0 { + panic("no return value specified for UpdateSchema") + } + + var r0 *commonpb.Status + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest) (*commonpb.Status, error)); ok { + return rf(ctx, req) + } + if rf, ok := ret.Get(0).(func(context.Context, *querypb.UpdateSchemaRequest) *commonpb.Status); ok { + r0 = rf(ctx, req) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*commonpb.Status) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *querypb.UpdateSchemaRequest) error); ok { + r1 = rf(ctx, req) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockWorker_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema' +type MockWorker_UpdateSchema_Call struct { + *mock.Call +} + +// UpdateSchema is a helper method to define mock.On call +// - ctx context.Context +// - req *querypb.UpdateSchemaRequest +func (_e *MockWorker_Expecter) UpdateSchema(ctx interface{}, req interface{}) *MockWorker_UpdateSchema_Call { + return &MockWorker_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema", ctx, req)} +} + +func (_c *MockWorker_UpdateSchema_Call) Run(run func(ctx context.Context, req *querypb.UpdateSchemaRequest)) *MockWorker_UpdateSchema_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*querypb.UpdateSchemaRequest)) + }) + return _c +} + +func (_c *MockWorker_UpdateSchema_Call) Return(_a0 *commonpb.Status, _a1 error) *MockWorker_UpdateSchema_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockWorker_UpdateSchema_Call) RunAndReturn(run func(context.Context, *querypb.UpdateSchemaRequest) (*commonpb.Status, error)) *MockWorker_UpdateSchema_Call { + _c.Call.Return(run) + return _c +} + // NewMockWorker creates a new instance of MockWorker. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. // The first argument is typically a *testing.T value. func NewMockWorker(t interface { diff --git a/internal/querynodev2/cluster/worker.go b/internal/querynodev2/cluster/worker.go index 6033a2014f..75994bf0ea 100644 --- a/internal/querynodev2/cluster/worker.go +++ b/internal/querynodev2/cluster/worker.go @@ -25,6 +25,7 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/streamrpc" "github.com/milvus-io/milvus/pkg/v2/log" @@ -44,6 +45,7 @@ type Worker interface { QuerySegments(ctx context.Context, req *querypb.QueryRequest) (*internalpb.RetrieveResults, error) QueryStreamSegments(ctx context.Context, req *querypb.QueryRequest, srv streamrpc.QueryStreamServer) error GetStatistics(ctx context.Context, req *querypb.GetStatisticsRequest) (*internalpb.GetStatisticsResponse, error) + UpdateSchema(ctx context.Context, req *querypb.UpdateSchemaRequest) (*commonpb.Status, error) IsHealthy() bool Stop() @@ -247,6 +249,11 @@ func (w *remoteWorker) GetStatistics(ctx context.Context, req *querypb.GetStatis return client.GetStatistics(ctx, req) } +func (w *remoteWorker) UpdateSchema(ctx context.Context, req *querypb.UpdateSchemaRequest) (*commonpb.Status, error) { + client := w.getClient() + return client.UpdateSchema(ctx, req) +} + func (w *remoteWorker) IsHealthy() bool { return true } diff --git a/internal/querynodev2/delegator/delegator.go b/internal/querynodev2/delegator/delegator.go index 7420ccc4e9..cb23d5c593 100644 --- a/internal/querynodev2/delegator/delegator.go +++ b/internal/querynodev2/delegator/delegator.go @@ -52,6 +52,7 @@ import ( "github.com/milvus-io/milvus/pkg/v2/mq/msgstream" "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "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/conc" "github.com/milvus-io/milvus/pkg/v2/util/funcutil" "github.com/milvus-io/milvus/pkg/v2/util/lifetime" @@ -76,6 +77,7 @@ type ShardDelegator interface { Query(ctx context.Context, req *querypb.QueryRequest) ([]*internalpb.RetrieveResults, error) QueryStream(ctx context.Context, req *querypb.QueryRequest, srv streamrpc.QueryStreamServer) error GetStatistics(ctx context.Context, req *querypb.GetStatisticsRequest) ([]*internalpb.GetStatisticsResponse, error) + UpdateSchema(ctx context.Context, sch *schemapb.CollectionSchema) error // data ProcessInsert(insertRecords map[int64]*InsertData) @@ -856,6 +858,51 @@ func (sd *shardDelegator) GetTSafe() uint64 { return sd.latestTsafe.Load() } +func (sd *shardDelegator) UpdateSchema(ctx context.Context, schema *schemapb.CollectionSchema) error { + log := sd.getLogger(ctx) + if err := sd.lifetime.Add(lifetime.IsWorking); err != nil { + return err + } + defer sd.lifetime.Done() + + log.Info("delegator received update schema event") + + sealed, growing, version, err := sd.distribution.PinReadableSegments() + if err != nil { + log.Warn("delegator failed to query, current distribution is not serviceable", zap.Error(err)) + return err + } + defer sd.distribution.Unpin(version) + + tasks, err := organizeSubTask(ctx, &querypb.UpdateSchemaRequest{ + Base: commonpbutil.NewMsgBase( + commonpbutil.WithSourceID(paramtable.GetNodeID()), + ), + CollectionID: sd.collectionID, + Schema: schema, + }, sealed, growing, sd, func(req *querypb.UpdateSchemaRequest, scope querypb.DataScope, segmentIDs []int64, targetID int64) *querypb.UpdateSchemaRequest { + nodeReq := typeutil.Clone(req) + nodeReq.GetBase().TargetID = targetID + return nodeReq + }) + if err != nil { + return err + } + + _, err = executeSubTasks(ctx, tasks, func(ctx context.Context, req *querypb.UpdateSchemaRequest, worker cluster.Worker) (*StatusWrapper, error) { + status, err := worker.UpdateSchema(ctx, req) + return (*StatusWrapper)(status), err + }, "UpdateSchema", log) + + return err +} + +type StatusWrapper commonpb.Status + +func (w *StatusWrapper) GetStatus() *commonpb.Status { + return (*commonpb.Status)(w) +} + // Close closes the delegator. func (sd *shardDelegator) Close() { sd.lifetime.SetState(lifetime.Stopped) diff --git a/internal/querynodev2/delegator/delegator_test.go b/internal/querynodev2/delegator/delegator_test.go index b32a040303..e91735a83e 100644 --- a/internal/querynodev2/delegator/delegator_test.go +++ b/internal/querynodev2/delegator/delegator_test.go @@ -1189,6 +1189,99 @@ func (s *DelegatorSuite) TestGetStats() { }) } +func (s *DelegatorSuite) TestUpdateSchema() { + s.delegator.Start() + paramtable.SetNodeID(1) + s.initSegments() + + s.Run("normal", func() { + workers := make(map[int64]*cluster.MockWorker) + worker1 := cluster.NewMockWorker(s.T()) + worker2 := cluster.NewMockWorker(s.T()) + + workers[1] = worker1 + workers[2] = worker2 + + worker1.EXPECT().UpdateSchema(mock.Anything, mock.AnythingOfType("*querypb.UpdateSchemaRequest")).RunAndReturn(func(ctx context.Context, usr *querypb.UpdateSchemaRequest) (*commonpb.Status, error) { + return merr.Success(), nil + }).Twice() + + worker2.EXPECT().UpdateSchema(mock.Anything, mock.AnythingOfType("*querypb.UpdateSchemaRequest")).RunAndReturn(func(ctx context.Context, usr *querypb.UpdateSchemaRequest) (*commonpb.Status, error) { + return merr.Success(), nil + }).Once() + + s.workerManager.EXPECT().GetWorker(mock.Anything, mock.AnythingOfType("int64")).Call.Return(func(_ context.Context, nodeID int64) cluster.Worker { + return workers[nodeID] + }, nil).Times(3) // currently node 1 will be called twice for growing & sealed + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + err := s.delegator.UpdateSchema(ctx, &schemapb.CollectionSchema{}) + s.NoError(err) + }) + + s.Run("worker_return_error", func() { + workers := make(map[int64]*cluster.MockWorker) + worker1 := cluster.NewMockWorker(s.T()) + worker2 := cluster.NewMockWorker(s.T()) + + workers[1] = worker1 + workers[2] = worker2 + + worker1.EXPECT().UpdateSchema(mock.Anything, mock.AnythingOfType("*querypb.UpdateSchemaRequest")).RunAndReturn(func(ctx context.Context, usr *querypb.UpdateSchemaRequest) (*commonpb.Status, error) { + return merr.Status(merr.WrapErrServiceInternal("mocked")), merr.WrapErrServiceInternal("mocked") + }).Maybe() + + worker2.EXPECT().UpdateSchema(mock.Anything, mock.AnythingOfType("*querypb.UpdateSchemaRequest")).RunAndReturn(func(ctx context.Context, usr *querypb.UpdateSchemaRequest) (*commonpb.Status, error) { + return merr.Success(), nil + }).Maybe() + + s.workerManager.EXPECT().GetWorker(mock.Anything, mock.AnythingOfType("int64")).Call.Return(func(_ context.Context, nodeID int64) cluster.Worker { + return workers[nodeID] + }, nil).Times(3) + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + err := s.delegator.UpdateSchema(ctx, &schemapb.CollectionSchema{}) + s.Error(err) + }) + + s.Run("worker_manager_error", func() { + s.workerManager.EXPECT().GetWorker(mock.Anything, mock.AnythingOfType("int64")).RunAndReturn(func(ctx context.Context, i int64) (cluster.Worker, error) { + return nil, merr.WrapErrServiceInternal("mocked") + }).Once() + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + err := s.delegator.UpdateSchema(ctx, &schemapb.CollectionSchema{}) + s.Error(err) + }) + + s.Run("distribution_not_serviceable", func() { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + sd, ok := s.delegator.(*shardDelegator) + s.Require().True(ok) + sd.distribution.AddOfflines(1001) + + err := s.delegator.UpdateSchema(ctx, &schemapb.CollectionSchema{}) + s.Error(err) + }) + + s.Run("cluster_not_serviceable", func() { + s.delegator.Close() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + err := s.delegator.UpdateSchema(ctx, &schemapb.CollectionSchema{}) + s.Error(err) + }) +} + func TestDelegatorSuite(t *testing.T) { suite.Run(t, new(DelegatorSuite)) } diff --git a/internal/querynodev2/delegator/mock_delegator.go b/internal/querynodev2/delegator/mock_delegator.go index 27df0b5493..513eaecc33 100644 --- a/internal/querynodev2/delegator/mock_delegator.go +++ b/internal/querynodev2/delegator/mock_delegator.go @@ -12,6 +12,8 @@ import ( querypb "github.com/milvus-io/milvus/pkg/v2/proto/querypb" + schemapb "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + streamrpc "github.com/milvus-io/milvus/internal/util/streamrpc" ) @@ -1105,6 +1107,53 @@ func (_c *MockShardDelegator_TryCleanExcludedSegments_Call) RunAndReturn(run fun return _c } +// UpdateSchema provides a mock function with given fields: ctx, sch +func (_m *MockShardDelegator) UpdateSchema(ctx context.Context, sch *schemapb.CollectionSchema) error { + ret := _m.Called(ctx, sch) + + if len(ret) == 0 { + panic("no return value specified for UpdateSchema") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, *schemapb.CollectionSchema) error); ok { + r0 = rf(ctx, sch) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockShardDelegator_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema' +type MockShardDelegator_UpdateSchema_Call struct { + *mock.Call +} + +// UpdateSchema is a helper method to define mock.On call +// - ctx context.Context +// - sch *schemapb.CollectionSchema +func (_e *MockShardDelegator_Expecter) UpdateSchema(ctx interface{}, sch interface{}) *MockShardDelegator_UpdateSchema_Call { + return &MockShardDelegator_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema", ctx, sch)} +} + +func (_c *MockShardDelegator_UpdateSchema_Call) Run(run func(ctx context.Context, sch *schemapb.CollectionSchema)) *MockShardDelegator_UpdateSchema_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*schemapb.CollectionSchema)) + }) + return _c +} + +func (_c *MockShardDelegator_UpdateSchema_Call) Return(_a0 error) *MockShardDelegator_UpdateSchema_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockShardDelegator_UpdateSchema_Call) RunAndReturn(run func(context.Context, *schemapb.CollectionSchema) error) *MockShardDelegator_UpdateSchema_Call { + _c.Call.Return(run) + return _c +} + // UpdateTSafe provides a mock function with given fields: ts func (_m *MockShardDelegator) UpdateTSafe(ts uint64) { _m.Called(ts) diff --git a/internal/querynodev2/local_worker.go b/internal/querynodev2/local_worker.go index 1571e82dfa..e6e2e226fb 100644 --- a/internal/querynodev2/local_worker.go +++ b/internal/querynodev2/local_worker.go @@ -19,6 +19,7 @@ package querynodev2 import ( "context" + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/querynodev2/cluster" "github.com/milvus-io/milvus/internal/util/streamrpc" "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" @@ -73,6 +74,10 @@ func (w *LocalWorker) GetStatistics(ctx context.Context, req *querypb.GetStatist return w.node.GetStatistics(ctx, req) } +func (w *LocalWorker) UpdateSchema(ctx context.Context, req *querypb.UpdateSchemaRequest) (*commonpb.Status, error) { + return w.node.UpdateSchema(ctx, req) +} + func (w *LocalWorker) IsHealthy() bool { return true } diff --git a/internal/querynodev2/pipeline/delete_node.go b/internal/querynodev2/pipeline/delete_node.go index aedda284d5..397041bd8f 100644 --- a/internal/querynodev2/pipeline/delete_node.go +++ b/internal/querynodev2/pipeline/delete_node.go @@ -17,6 +17,7 @@ package pipeline import ( + "context" "fmt" "github.com/samber/lo" @@ -76,6 +77,10 @@ func (dNode *deleteNode) Operate(in Msg) Msg { dNode.delegator.ProcessDelete(lo.Values(deleteDatas), nodeMsg.timeRange.timestampMax) } + if nodeMsg.schema != nil { + dNode.delegator.UpdateSchema(context.Background(), nodeMsg.schema) + } + // update tSafe dNode.delegator.UpdateTSafe(nodeMsg.timeRange.timestampMax) return nil diff --git a/internal/querynodev2/pipeline/filter_node.go b/internal/querynodev2/pipeline/filter_node.go index cdf87002ff..72b61349f0 100644 --- a/internal/querynodev2/pipeline/filter_node.go +++ b/internal/querynodev2/pipeline/filter_node.go @@ -29,6 +29,7 @@ import ( "github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/metrics" "github.com/milvus-io/milvus/pkg/v2/mq/msgstream" + "github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor" "github.com/milvus-io/milvus/pkg/v2/util/merr" "github.com/milvus-io/milvus/pkg/v2/util/paramtable" "github.com/milvus-io/milvus/pkg/v2/util/tsoutil" @@ -134,6 +135,13 @@ func (fNode *filterNode) filtrate(c *Collection, msg msgstream.TsMsg) error { return err } } + case commonpb.MsgType_AddCollectionField: + schemaMsg := msg.(*adaptor.SchemaChangeMessageBody) + header := schemaMsg.SchemaChangeMessage.Header() + if header.GetCollectionId() != fNode.collectionID { + return merr.WrapErrCollectionNotFound(header.GetCollectionId()) + } + return nil default: return merr.WrapErrParameterInvalid("msgType is Insert or Delete", "not") } diff --git a/internal/querynodev2/pipeline/insert_node.go b/internal/querynodev2/pipeline/insert_node.go index 264928f253..1c34ec58db 100644 --- a/internal/querynodev2/pipeline/insert_node.go +++ b/internal/querynodev2/pipeline/insert_node.go @@ -118,6 +118,7 @@ func (iNode *insertNode) Operate(in Msg) Msg { return &deleteNodeMsg{ deleteMsgs: nodeMsg.deleteMsgs, timeRange: nodeMsg.timeRange, + schema: nodeMsg.schema, } } diff --git a/internal/querynodev2/pipeline/message.go b/internal/querynodev2/pipeline/message.go index f8ef5a128f..e88b4f896c 100644 --- a/internal/querynodev2/pipeline/message.go +++ b/internal/querynodev2/pipeline/message.go @@ -18,9 +18,11 @@ package pipeline import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/querynodev2/collector" "github.com/milvus-io/milvus/internal/querynodev2/delegator" "github.com/milvus-io/milvus/pkg/v2/mq/msgstream" + "github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor" "github.com/milvus-io/milvus/pkg/v2/util/merr" "github.com/milvus-io/milvus/pkg/v2/util/metricsinfo" ) @@ -30,11 +32,13 @@ type insertNodeMsg struct { deleteMsgs []*DeleteMsg insertDatas map[int64]*delegator.InsertData timeRange TimeRange + schema *schemapb.CollectionSchema } type deleteNodeMsg struct { deleteMsgs []*DeleteMsg timeRange TimeRange + schema *schemapb.CollectionSchema } func (msg *insertNodeMsg) append(taskMsg msgstream.TsMsg) error { @@ -47,6 +51,13 @@ func (msg *insertNodeMsg) append(taskMsg msgstream.TsMsg) error { deleteMsg := taskMsg.(*DeleteMsg) msg.deleteMsgs = append(msg.deleteMsgs, deleteMsg) collector.Rate.Add(metricsinfo.DeleteConsumeThroughput, float64(deleteMsg.Size())) + case commonpb.MsgType_AddCollectionField: + schemaMsg := taskMsg.(*adaptor.SchemaChangeMessageBody) + body, err := schemaMsg.SchemaChangeMessage.Body() + if err != nil { + return err + } + msg.schema = body.GetSchema() default: return merr.WrapErrParameterInvalid("msgType is Insert or Delete", "not") } diff --git a/internal/querynodev2/segments/collection.go b/internal/querynodev2/segments/collection.go index b968eb22b8..8e4bbc77bd 100644 --- a/internal/querynodev2/segments/collection.go +++ b/internal/querynodev2/segments/collection.go @@ -33,6 +33,7 @@ import ( "github.com/milvus-io/milvus/pkg/v2/metrics" "github.com/milvus-io/milvus/pkg/v2/proto/querypb" "github.com/milvus-io/milvus/pkg/v2/proto/segcorepb" + "github.com/milvus-io/milvus/pkg/v2/util/merr" "github.com/milvus-io/milvus/pkg/v2/util/paramtable" "github.com/milvus-io/milvus/pkg/v2/util/typeutil" ) @@ -47,6 +48,8 @@ type CollectionManager interface { // returns true if the collection ref count goes 0, or the collection not exists, // return false otherwise Unref(collectionID int64, count uint32) bool + // UpdateSchema update the underlying collection schema of the provided collection. + UpdateSchema(collectionID int64, schema *schemapb.CollectionSchema) error } type collectionManager struct { @@ -106,6 +109,22 @@ func (m *collectionManager) PutOrRef(collectionID int64, schema *schemapb.Collec return nil } +func (m *collectionManager) UpdateSchema(collectionID int64, schema *schemapb.CollectionSchema) error { + m.mut.Lock() + defer m.mut.Unlock() + + collection, ok := m.collections[collectionID] + if !ok { + return merr.WrapErrCollectionNotFound(collectionID, "collection not found in querynode collection manager") + } + + if err := collection.ccollection.UpdateSchema(schema); err != nil { + return err + } + collection.schema.Store(schema) + return nil +} + func (m *collectionManager) updateMetric() { metrics.QueryNodeNumCollections.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Set(float64(len(m.collections))) } diff --git a/internal/querynodev2/segments/collection_test.go b/internal/querynodev2/segments/collection_test.go new file mode 100644 index 0000000000..db0f1fc6a5 --- /dev/null +++ b/internal/querynodev2/segments/collection_test.go @@ -0,0 +1,93 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package segments + +import ( + "testing" + + "github.com/stretchr/testify/suite" + + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/mocks/util/mock_segcore" + "github.com/milvus-io/milvus/pkg/v2/common" + "github.com/milvus-io/milvus/pkg/v2/proto/querypb" + "github.com/milvus-io/milvus/pkg/v2/util/paramtable" +) + +type CollectionManagerSuite struct { + suite.Suite + cm *collectionManager +} + +func (s *CollectionManagerSuite) SetupSuite() { + paramtable.Init() +} + +func (s *CollectionManagerSuite) SetupTest() { + s.cm = NewCollectionManager() + schema := mock_segcore.GenTestCollectionSchema("collection_1", schemapb.DataType_Int64, false) + err := s.cm.PutOrRef(1, schema, mock_segcore.GenTestIndexMeta(1, schema), &querypb.LoadMetaInfo{ + LoadType: querypb.LoadType_LoadCollection, + DbProperties: []*commonpb.KeyValuePair{ + { + Key: common.ReplicateIDKey, + Value: "local-test", + }, + }, + }) + s.Require().NoError(err) +} + +func (s *CollectionManagerSuite) TestUpdateSchema() { + s.Run("normal_case", func() { + schema := mock_segcore.GenTestCollectionSchema("collection_1", schemapb.DataType_Int64, false) + schema.Fields = append(schema.Fields, &schemapb.FieldSchema{ + FieldID: common.StartOfUserFieldID + int64(len(schema.Fields)), + Name: "added_field", + DataType: schemapb.DataType_Bool, + Nullable: true, + }) + + err := s.cm.UpdateSchema(1, schema) + s.NoError(err) + }) + + s.Run("not_exist_collection", func() { + schema := mock_segcore.GenTestCollectionSchema("collection_1", schemapb.DataType_Int64, false) + schema.Fields = append(schema.Fields, &schemapb.FieldSchema{ + FieldID: common.StartOfUserFieldID + int64(len(schema.Fields)), + Name: "added_field", + DataType: schemapb.DataType_Bool, + Nullable: true, + }) + + err := s.cm.UpdateSchema(2, schema) + s.Error(err) + }) + + s.Run("nil_schema", func() { + s.NotPanics(func() { + err := s.cm.UpdateSchema(1, nil) + s.Error(err) + }) + }) +} + +func TestCollectionManager(t *testing.T) { + suite.Run(t, new(CollectionManagerSuite)) +} diff --git a/internal/querynodev2/segments/mock_collection_manager.go b/internal/querynodev2/segments/mock_collection_manager.go index fe8368a514..5baf3f5f22 100644 --- a/internal/querynodev2/segments/mock_collection_manager.go +++ b/internal/querynodev2/segments/mock_collection_manager.go @@ -308,6 +308,53 @@ func (_c *MockCollectionManager_Unref_Call) RunAndReturn(run func(int64, uint32) return _c } +// UpdateSchema provides a mock function with given fields: collectionID, schema +func (_m *MockCollectionManager) UpdateSchema(collectionID int64, schema *schemapb.CollectionSchema) error { + ret := _m.Called(collectionID, schema) + + if len(ret) == 0 { + panic("no return value specified for UpdateSchema") + } + + var r0 error + if rf, ok := ret.Get(0).(func(int64, *schemapb.CollectionSchema) error); ok { + r0 = rf(collectionID, schema) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockCollectionManager_UpdateSchema_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'UpdateSchema' +type MockCollectionManager_UpdateSchema_Call struct { + *mock.Call +} + +// UpdateSchema is a helper method to define mock.On call +// - collectionID int64 +// - schema *schemapb.CollectionSchema +func (_e *MockCollectionManager_Expecter) UpdateSchema(collectionID interface{}, schema interface{}) *MockCollectionManager_UpdateSchema_Call { + return &MockCollectionManager_UpdateSchema_Call{Call: _e.mock.On("UpdateSchema", collectionID, schema)} +} + +func (_c *MockCollectionManager_UpdateSchema_Call) Run(run func(collectionID int64, schema *schemapb.CollectionSchema)) *MockCollectionManager_UpdateSchema_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(int64), args[1].(*schemapb.CollectionSchema)) + }) + return _c +} + +func (_c *MockCollectionManager_UpdateSchema_Call) Return(_a0 error) *MockCollectionManager_UpdateSchema_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockCollectionManager_UpdateSchema_Call) RunAndReturn(run func(int64, *schemapb.CollectionSchema) error) *MockCollectionManager_UpdateSchema_Call { + _c.Call.Return(run) + return _c +} + // NewMockCollectionManager creates a new instance of MockCollectionManager. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. // The first argument is typically a *testing.T value. func NewMockCollectionManager(t interface { diff --git a/internal/querynodev2/services.go b/internal/querynodev2/services.go index 4eb969c1c7..63eff8cfae 100644 --- a/internal/querynodev2/services.go +++ b/internal/querynodev2/services.go @@ -505,6 +505,28 @@ func (node *QueryNode) LoadSegments(ctx context.Context, req *querypb.LoadSegmen return merr.Success(), nil } +// UpdateSchema updates the schema of the collection on the querynode. +func (node *QueryNode) UpdateSchema(ctx context.Context, req *querypb.UpdateSchemaRequest) (*commonpb.Status, error) { + defer node.updateDistributionModifyTS() + + // check node healthy + if err := node.lifetime.Add(merr.IsHealthy); err != nil { + return merr.Status(err), nil + } + defer node.lifetime.Done() + + log := log.Ctx(ctx).With( + zap.Int64("collectionID", req.GetCollectionID()), + ) + + err := node.manager.Collection.UpdateSchema(req.GetCollectionID(), req.GetSchema()) + if err != nil { + log.Warn("failed to update schema", zap.Error(err)) + } + + return merr.Status(err), nil +} + // ReleaseCollection clears all data related to this collection on the querynode func (node *QueryNode) ReleaseCollection(ctx context.Context, in *querypb.ReleaseCollectionRequest) (*commonpb.Status, error) { if err := node.lifetime.Add(merr.IsHealthyOrStopping); err != nil { diff --git a/internal/querynodev2/services_test.go b/internal/querynodev2/services_test.go index bfb3f40218..bced48222a 100644 --- a/internal/querynodev2/services_test.go +++ b/internal/querynodev2/services_test.go @@ -2334,6 +2334,44 @@ func (suite *ServiceSuite) TestLoadPartition() { suite.Equal(commonpb.ErrorCode_Success, status.ErrorCode) } +func (suite *ServiceSuite) TestUpdateSchema() { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + req := &querypb.UpdateSchemaRequest{ + CollectionID: suite.collectionID, + Schema: suite.schema, + } + manager := suite.node.manager.Collection + // reset manager to align default teardown logic + defer func() { + suite.node.manager.Collection = manager + }() + mockManager := segments.NewMockCollectionManager(suite.T()) + suite.node.manager.Collection = mockManager + + suite.Run("normal", func() { + mockManager.EXPECT().UpdateSchema(suite.collectionID, suite.schema).Return(nil).Once() + + status, err := suite.node.UpdateSchema(ctx, req) + suite.NoError(merr.CheckRPCCall(status, err)) + }) + + suite.Run("manager_returns_error", func() { + mockManager.EXPECT().UpdateSchema(suite.collectionID, suite.schema).Return(merr.WrapErrServiceInternal("mocked")).Once() + + status, err := suite.node.UpdateSchema(ctx, req) + suite.Error(merr.CheckRPCCall(status, err)) + }) + + suite.Run("abonormal_node", func() { + suite.node.UpdateStateCode(commonpb.StateCode_Abnormal) + defer suite.node.UpdateStateCode(commonpb.StateCode_Healthy) + status, err := suite.node.UpdateSchema(ctx, req) + suite.Error(merr.CheckRPCCall(status, err)) + }) +} + func TestQueryNodeService(t *testing.T) { suite.Run(t, new(ServiceSuite)) } diff --git a/internal/rootcoord/add_field_task.go b/internal/rootcoord/add_field_task.go index 62fc00ded6..b9fedbe829 100644 --- a/internal/rootcoord/add_field_task.go +++ b/internal/rootcoord/add_field_task.go @@ -88,8 +88,9 @@ func executeAddCollectionFieldTaskSteps(ctx context.Context, req *milvuspb.AddCollectionFieldRequest, ts Timestamp, ) error { - oldColl := col.Clone() redoTask := newBaseRedoTask(core.stepExecutor) + + oldColl := col.Clone() redoTask.AddSyncStep(&AddCollectionFieldStep{ baseStep: baseStep{core: core}, oldColl: oldColl, @@ -97,6 +98,14 @@ func executeAddCollectionFieldTaskSteps(ctx context.Context, ts: ts, }) + updatedCollection := col.Clone() + updatedCollection.Fields = append(updatedCollection.Fields, newField) + redoTask.AddSyncStep(&WriteSchemaChangeWALStep{ + baseStep: baseStep{core: core}, + collection: updatedCollection, + ts: ts, + }) + req.CollectionID = oldColl.CollectionID redoTask.AddSyncStep(&BroadcastAlteredCollectionStep{ baseStep: baseStep{core: core}, diff --git a/internal/rootcoord/add_field_task_test.go b/internal/rootcoord/add_field_task_test.go index 418555e63c..e6f1fdb727 100644 --- a/internal/rootcoord/add_field_task_test.go +++ b/internal/rootcoord/add_field_task_test.go @@ -28,8 +28,11 @@ 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/distributed/streaming" "github.com/milvus-io/milvus/internal/metastore/model" + "github.com/milvus-io/milvus/internal/mocks/distributed/mock_streaming" mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks" + "github.com/milvus-io/milvus/pkg/v2/streaming/util/types" "github.com/milvus-io/milvus/pkg/v2/util/merr" ) @@ -74,7 +77,29 @@ func Test_AddCollectionFieldTask_Prepare(t *testing.T) { } func Test_AddCollectionFieldTask_Execute(t *testing.T) { - t.Run("failed to get collection", func(t *testing.T) { + b := mock_streaming.NewMockBroadcast(t) + wal := mock_streaming.NewMockWALAccesser(t) + wal.EXPECT().Broadcast().Return(b).Maybe() + streaming.SetWALForTest(wal) + + testCollection := &model.Collection{ + CollectionID: int64(1), + Fields: []*model.Field{ + { + Name: "pk", + IsPrimaryKey: true, + DataType: schemapb.DataType_Int64, + }, + { + Name: "vec", + DataType: schemapb.DataType_FloatVector, + }, + }, + PhysicalChannelNames: []string{"dml_ch_01", "dml_ch_02"}, + VirtualChannelNames: []string{"dml_ch_01", "dml_ch_02"}, + } + + t.Run("failed_to_get_collection", func(t *testing.T) { metaTable := mockrootcoord.NewIMetaTable(t) metaTable.EXPECT().GetCollectionByName(mock.Anything, mock.Anything, "not_existed_coll", mock.Anything).Return(nil, merr.WrapErrCollectionNotFound("not_existed_coll")) core := newTestCore(withMeta(metaTable)) @@ -86,34 +111,24 @@ func Test_AddCollectionFieldTask_Execute(t *testing.T) { }, } err := task.Execute(context.Background()) - assert.Error(t, err) + assert.Error(t, err, "error shall be return when get collection failed") }) t.Run("add field step failed", func(t *testing.T) { meta := mockrootcoord.NewIMetaTable(t) - meta.On("GetCollectionByName", + meta.EXPECT().GetCollectionByName( mock.Anything, mock.Anything, mock.Anything, mock.Anything, - ).Return(&model.Collection{CollectionID: int64(1), Fields: []*model.Field{ - { - Name: "pk", - IsPrimaryKey: true, - DataType: schemapb.DataType_Int64, - }, - { - Name: "vec", - DataType: schemapb.DataType_FloatVector, - }, - }}, nil) - meta.On("AlterCollection", + ).Return(testCollection, nil) + meta.EXPECT().AlterCollection( mock.Anything, mock.Anything, mock.Anything, mock.Anything, ).Return(errors.New("mock")) - meta.On("ListAliasesByID", mock.Anything, mock.Anything).Return([]string{}) + meta.EXPECT().ListAliasesByID(mock.Anything, mock.Anything).Return([]string{}) alloc := newMockIDAllocator() core := newTestCore(withValidProxyManager(), withMeta(meta), withIDAllocator(alloc)) task := &addCollectionFieldTask{ @@ -132,31 +147,74 @@ func Test_AddCollectionFieldTask_Execute(t *testing.T) { assert.Error(t, err) }) - t.Run("broadcast add field step failed", func(t *testing.T) { + t.Run("write_wal_fail", func(t *testing.T) { meta := mockrootcoord.NewIMetaTable(t) - meta.On("GetCollectionByName", + meta.EXPECT().GetCollectionByName( mock.Anything, mock.Anything, mock.Anything, mock.Anything, - ).Return(&model.Collection{CollectionID: int64(1), Fields: []*model.Field{ - { - Name: "pk", - IsPrimaryKey: true, - DataType: schemapb.DataType_Int64, - }, - { - Name: "vec", - DataType: schemapb.DataType_FloatVector, - }, - }}, nil) - meta.On("AlterCollection", + ).Return(testCollection, nil) + meta.EXPECT().AlterCollection( mock.Anything, mock.Anything, mock.Anything, mock.Anything, ).Return(nil) - meta.On("ListAliasesByID", mock.Anything, mock.Anything).Return([]string{}) + meta.EXPECT().ListAliasesByID(mock.Anything, mock.Anything).Return([]string{}) + + broker := newMockBroker() + broker.BroadcastAlteredCollectionFunc = func(ctx context.Context, req *milvuspb.AlterCollectionRequest) error { + return errors.New("mock") + } + alloc := newMockIDAllocator() + core := newTestCore(withValidProxyManager(), withMeta(meta), withBroker(broker), withIDAllocator(alloc)) + task := &addCollectionFieldTask{ + baseTask: newBaseTask(context.Background(), core), + Req: &milvuspb.AddCollectionFieldRequest{ + Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterAlias}, + CollectionName: "coll", + }, + fieldSchema: &schemapb.FieldSchema{ + Name: "fid", + DataType: schemapb.DataType_Bool, + Nullable: true, + }, + } + + t.Run("write_schema_change_fail", func(t *testing.T) { + b.EXPECT().Append(mock.Anything, mock.Anything).Return(nil, merr.WrapErrServiceInternal("mocked")).Once() + + err := task.Execute(context.Background()) + assert.Error(t, err) + }) + + t.Run("write_flush_failed", func(t *testing.T) { + b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil).Once() + b.EXPECT().Append(mock.Anything, mock.Anything).Return(nil, merr.WrapErrServiceInternal("mocked")).Once() + + err := task.Execute(context.Background()) + assert.Error(t, err) + }) + }) + + t.Run("broadcast add field step failed", func(t *testing.T) { + b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil).Times(2) + + meta := mockrootcoord.NewIMetaTable(t) + meta.EXPECT().GetCollectionByName( + mock.Anything, + mock.Anything, + mock.Anything, + mock.Anything, + ).Return(testCollection, nil) + meta.EXPECT().AlterCollection( + mock.Anything, + mock.Anything, + mock.Anything, + mock.Anything, + ).Return(nil) + meta.EXPECT().ListAliasesByID(mock.Anything, mock.Anything).Return([]string{}) broker := newMockBroker() broker.BroadcastAlteredCollectionFunc = func(ctx context.Context, req *milvuspb.AlterCollectionRequest) error { @@ -181,30 +239,22 @@ func Test_AddCollectionFieldTask_Execute(t *testing.T) { }) t.Run("expire cache failed", func(t *testing.T) { + b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil).Times(2) + meta := mockrootcoord.NewIMetaTable(t) - meta.On("GetCollectionByName", + meta.EXPECT().GetCollectionByName( mock.Anything, mock.Anything, mock.Anything, mock.Anything, - ).Return(&model.Collection{CollectionID: int64(1), Fields: []*model.Field{ - { - Name: "pk", - IsPrimaryKey: true, - DataType: schemapb.DataType_Int64, - }, - { - Name: "vec", - DataType: schemapb.DataType_FloatVector, - }, - }}, nil) - meta.On("AlterCollection", + ).Return(testCollection, nil) + meta.EXPECT().AlterCollection( mock.Anything, mock.Anything, mock.Anything, mock.Anything, ).Return(nil) - meta.On("ListAliasesByID", mock.Anything, mock.Anything).Return([]string{}) + meta.EXPECT().ListAliasesByID(mock.Anything, mock.Anything).Return([]string{}) broker := newMockBroker() broker.BroadcastAlteredCollectionFunc = func(ctx context.Context, req *milvuspb.AlterCollectionRequest) error { @@ -230,30 +280,22 @@ func Test_AddCollectionFieldTask_Execute(t *testing.T) { }) t.Run("normal case", func(t *testing.T) { + b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil).Times(2) + meta := mockrootcoord.NewIMetaTable(t) - meta.On("GetCollectionByName", + meta.EXPECT().GetCollectionByName( mock.Anything, mock.Anything, mock.Anything, mock.Anything, - ).Return(&model.Collection{CollectionID: int64(1), Fields: []*model.Field{ - { - Name: "pk", - IsPrimaryKey: true, - DataType: schemapb.DataType_Int64, - }, - { - Name: "vec", - DataType: schemapb.DataType_FloatVector, - }, - }}, nil) - meta.On("AlterCollection", + ).Return(testCollection, nil) + meta.EXPECT().AlterCollection( mock.Anything, mock.Anything, mock.Anything, mock.Anything, ).Return(nil) - meta.On("ListAliasesByID", mock.Anything, mock.Anything).Return([]string{}) + meta.EXPECT().ListAliasesByID(mock.Anything, mock.Anything).Return([]string{}) broker := newMockBroker() broker.BroadcastAlteredCollectionFunc = func(ctx context.Context, req *milvuspb.AlterCollectionRequest) error { @@ -264,7 +306,7 @@ func Test_AddCollectionFieldTask_Execute(t *testing.T) { task := &addCollectionFieldTask{ baseTask: newBaseTask(context.Background(), core), Req: &milvuspb.AddCollectionFieldRequest{ - Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AlterAlias}, + Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_AddCollectionField}, CollectionName: "coll", }, fieldSchema: &schemapb.FieldSchema{ diff --git a/internal/rootcoord/step.go b/internal/rootcoord/step.go index 0dd43fad9f..dbb719a3ac 100644 --- a/internal/rootcoord/step.go +++ b/internal/rootcoord/step.go @@ -26,11 +26,13 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/distributed/streaming" "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/pkg/v2/log" pb "github.com/milvus-io/milvus/pkg/v2/proto/etcdpb" + "github.com/milvus-io/milvus/pkg/v2/proto/messagespb" "github.com/milvus-io/milvus/pkg/v2/streaming/util/message" "github.com/milvus-io/milvus/pkg/v2/util/commonpbutil" ) @@ -514,6 +516,77 @@ func (a *AddCollectionFieldStep) Desc() string { return fmt.Sprintf("add field, collectionID: %d, fieldID: %d, ts: %d", a.oldColl.CollectionID, a.newField.FieldID, a.ts) } +type WriteSchemaChangeWALStep struct { + baseStep + collection *model.Collection + ts Timestamp +} + +func (s *WriteSchemaChangeWALStep) Execute(ctx context.Context) ([]nestedStep, error) { + vchannels := s.collection.VirtualChannelNames + schema := &schemapb.CollectionSchema{ + Name: s.collection.Name, + Description: s.collection.Description, + AutoID: s.collection.AutoID, + Fields: model.MarshalFieldModels(s.collection.Fields), + Functions: model.MarshalFunctionModels(s.collection.Functions), + EnableDynamicField: s.collection.EnableDynamicField, + Properties: s.collection.Properties, + } + + schemaMsg, err := message.NewSchemaChangeMessageBuilderV2(). + WithBroadcast(vchannels). + WithHeader(&message.SchemaChangeMessageHeader{ + CollectionId: s.collection.CollectionID, + }). + WithBody(&message.SchemaChangeMessageBody{ + Schema: schema, + ModifyTs: s.ts, + }).BuildBroadcast() + if err != nil { + return nil, err + } + + resp, err := streaming.WAL().Broadcast().Append(ctx, schemaMsg) + if err != nil { + return nil, err + } + + log.Ctx(ctx).Info( + "broadcast schema change success", + zap.Uint64("broadcastID", resp.BroadcastID), + zap.Any("appendResults", resp.AppendResults), + ) + + flushMsg, err := message.NewManualFlushMessageBuilderV2(). + WithBroadcast(vchannels). + WithHeader(&messagespb.ManualFlushMessageHeader{ + CollectionId: s.collection.CollectionID, + FlushTs: s.ts, + }). + WithBody(&message.ManualFlushMessageBody{}).BuildBroadcast() + if err != nil { + return nil, err + } + + resp, err = streaming.WAL().Broadcast().Append(ctx, flushMsg) + if err != nil { + return nil, err + } + + log.Ctx(ctx).Info( + "broadcast schema change success", + zap.Uint64("broadcastID", resp.BroadcastID), + zap.Any("appendResults", resp.AppendResults), + ) + + return nil, nil +} + +func (s *WriteSchemaChangeWALStep) Desc() string { + return fmt.Sprintf("write schema change WALcollectionID: %d, ts: %d", s.collection.CollectionID, s.ts) +} + type AlterDatabaseStep struct { baseStep oldDB *model.Database diff --git a/internal/streamingnode/server/flusher/flusherimpl/msg_handler_impl.go b/internal/streamingnode/server/flusher/flusherimpl/msg_handler_impl.go index ff9c23719d..6aa55e6699 100644 --- a/internal/streamingnode/server/flusher/flusherimpl/msg_handler_impl.go +++ b/internal/streamingnode/server/flusher/flusherimpl/msg_handler_impl.go @@ -31,6 +31,7 @@ import ( "github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/proto/internalpb" "github.com/milvus-io/milvus/pkg/v2/streaming/util/message" + "github.com/milvus-io/milvus/pkg/v2/streaming/util/message/adaptor" "github.com/milvus-io/milvus/pkg/v2/streaming/util/types" "github.com/milvus-io/milvus/pkg/v2/util/funcutil" "github.com/milvus-io/milvus/pkg/v2/util/merr" @@ -84,7 +85,21 @@ func (impl *msgHandlerImpl) HandleManualFlush(vchannel string, flushMsg message. if err := impl.wbMgr.FlushChannel(context.Background(), vchannel, flushMsg.Header().GetFlushTs()); err != nil { return errors.Wrap(err, "failed to flush channel") } - return nil + broadcastID := flushMsg.BroadcastHeader().BroadcastID + if broadcastID == 0 { + return nil + } + return streaming.WAL().Broadcast().Ack(context.Background(), types.BroadcastAckRequest{ + BroadcastID: flushMsg.BroadcastHeader().BroadcastID, + VChannel: vchannel, + }) +} + +func (impl *msgHandlerImpl) HandleSchemaChange(ctx context.Context, vchannel string, msg *adaptor.SchemaChangeMessageBody) error { + return streaming.WAL().Broadcast().Ack(ctx, types.BroadcastAckRequest{ + BroadcastID: msg.BroadcastID, + VChannel: vchannel, + }) } func (impl *msgHandlerImpl) HandleImport(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg) error { diff --git a/internal/streamingnode/server/wal/RAEDME.md b/internal/streamingnode/server/wal/README.md similarity index 100% rename from internal/streamingnode/server/wal/RAEDME.md rename to internal/streamingnode/server/wal/README.md diff --git a/internal/streamingnode/server/wal/adaptor/initializing.go b/internal/streamingnode/server/wal/adaptor/initializing.go index 4cb62094e0..1a39700a9c 100644 --- a/internal/streamingnode/server/wal/adaptor/initializing.go +++ b/internal/streamingnode/server/wal/adaptor/initializing.go @@ -29,7 +29,7 @@ func buildInterceptorParams(ctx context.Context, underlyingWALImpls walimpls.WAL capacity := int(paramtable.Get().StreamingCfg.WALWriteAheadBufferCapacity.GetAsSize()) keepalive := paramtable.Get().StreamingCfg.WALWriteAheadBufferKeepalive.GetAsDurationByParse() - writeAheadBuffer := wab.NewWirteAheadBuffer( + writeAheadBuffer := wab.NewWriteAheadBuffer( underlyingWALImpls.Channel().Name, resource.Resource().Logger().With(), capacity, diff --git a/internal/streamingnode/server/wal/interceptors/timetick/timetick_sync_operator_test.go b/internal/streamingnode/server/wal/interceptors/timetick/timetick_sync_operator_test.go index 1346e03e64..0cc1080bdc 100644 --- a/internal/streamingnode/server/wal/interceptors/timetick/timetick_sync_operator_test.go +++ b/internal/streamingnode/server/wal/interceptors/timetick/timetick_sync_operator_test.go @@ -48,7 +48,7 @@ func TestTimeTickSyncOperator(t *testing.T) { WAL: walFuture, InitializedTimeTick: ts, InitializedMessageID: msgID, - WriteAheadBuffer: wab.NewWirteAheadBuffer( + WriteAheadBuffer: wab.NewWriteAheadBuffer( channel.Name, resource.Resource().Logger().With(), 1024, diff --git a/internal/streamingnode/server/wal/interceptors/wab/write_ahead_buffer.go b/internal/streamingnode/server/wal/interceptors/wab/write_ahead_buffer.go index fcf0cc5008..97e1c5caa7 100644 --- a/internal/streamingnode/server/wal/interceptors/wab/write_ahead_buffer.go +++ b/internal/streamingnode/server/wal/interceptors/wab/write_ahead_buffer.go @@ -24,7 +24,7 @@ type ROWriteAheadBuffer interface { } // NewWriteAheadBuffer creates a new WriteAheadBuffer. -func NewWirteAheadBuffer( +func NewWriteAheadBuffer( pchannel string, logger *log.MLogger, capacity int, diff --git a/internal/streamingnode/server/wal/interceptors/wab/write_ahead_buffer_test.go b/internal/streamingnode/server/wal/interceptors/wab/write_ahead_buffer_test.go index 0a80f16f23..05f836a817 100644 --- a/internal/streamingnode/server/wal/interceptors/wab/write_ahead_buffer_test.go +++ b/internal/streamingnode/server/wal/interceptors/wab/write_ahead_buffer_test.go @@ -17,7 +17,7 @@ import ( func TestWriteAheadBufferWithOnlyTrivialTimeTick(t *testing.T) { ctx := context.Background() - wb := NewWirteAheadBuffer("pchannel", log.With(), 5*1024*1024, 30*time.Second, createTimeTickMessage(0)) + wb := NewWriteAheadBuffer("pchannel", log.With(), 5*1024*1024, 30*time.Second, createTimeTickMessage(0)) // Test timeout ctx, cancel := context.WithTimeout(ctx, 1*time.Millisecond) @@ -81,7 +81,7 @@ func TestWriteAheadBufferWithOnlyTrivialTimeTick(t *testing.T) { func TestWriteAheadBuffer(t *testing.T) { // Concurrent add message into bufffer and make syncup. // The reader should never lost any message if no eviction happen. - wb := NewWirteAheadBuffer("pchannel", log.With(), 5*1024*1024, 30*time.Second, createTimeTickMessage(1)) + wb := NewWriteAheadBuffer("pchannel", log.With(), 5*1024*1024, 30*time.Second, createTimeTickMessage(1)) expectedLastTimeTick := uint64(10000) ch := make(chan struct{}) totalCnt := 0 @@ -183,7 +183,7 @@ func TestWriteAheadBuffer(t *testing.T) { } func TestWriteAheadBufferEviction(t *testing.T) { - wb := NewWirteAheadBuffer("pchannel", log.With(), 5*1024*1024, 50*time.Millisecond, createTimeTickMessage(0)) + wb := NewWriteAheadBuffer("pchannel", log.With(), 5*1024*1024, 50*time.Millisecond, createTimeTickMessage(0)) msgs := make([]message.ImmutableMessage, 0) for i := 1; i < 100; i++ { diff --git a/internal/util/mock/grpc_querynode_client.go b/internal/util/mock/grpc_querynode_client.go index 60a1069e0b..a78fdaaeee 100644 --- a/internal/util/mock/grpc_querynode_client.go +++ b/internal/util/mock/grpc_querynode_client.go @@ -134,6 +134,10 @@ func (m *GrpcQueryNodeClient) DeleteBatch(ctx context.Context, in *querypb.Delet return &querypb.DeleteBatchResponse{}, m.Err } +func (m *GrpcQueryNodeClient) UpdateSchema(ctx context.Context, in *querypb.UpdateSchemaRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + return &commonpb.Status{}, m.Err +} + func (m *GrpcQueryNodeClient) Close() error { return m.Err } diff --git a/internal/util/pipeline/stream_pipeline.go b/internal/util/pipeline/stream_pipeline.go index 88b9c98854..a9800a3941 100644 --- a/internal/util/pipeline/stream_pipeline.go +++ b/internal/util/pipeline/stream_pipeline.go @@ -114,7 +114,8 @@ func (p *streamPipeline) ConsumeMsgStream(ctx context.Context, position *msgpb.M // only consume messages with timestamp >= position timestamp options.DeliverFilterTimeTickGTE(position.GetTimestamp()), // only consume insert and delete messages - options.DeliverFilterMessageType(message.MessageTypeInsert, message.MessageTypeDelete), + // also schema change message to notify schema change events + options.DeliverFilterMessageType(message.MessageTypeInsert, message.MessageTypeDelete, message.MessageTypeSchemaChange), }, MessageHandler: handler, }) diff --git a/internal/util/segcore/collection.go b/internal/util/segcore/collection.go index 94bea2c4fd..e3e60f074c 100644 --- a/internal/util/segcore/collection.go +++ b/internal/util/segcore/collection.go @@ -16,6 +16,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/pkg/v2/proto/segcorepb" + "github.com/milvus-io/milvus/pkg/v2/util/merr" ) // CreateCCollectionRequest is a request to create a CCollection. @@ -85,6 +86,20 @@ func (c *CCollection) IndexMeta() *segcorepb.CollectionIndexMeta { return c.indexMeta } +func (c *CCollection) UpdateSchema(sch *schemapb.CollectionSchema) error { + if sch == nil { + return merr.WrapErrServiceInternal("update collection schema with nil") + } + + schemaBlob, err := proto.Marshal(sch) + if err != nil { + return err + } + + status := C.UpdateSchema(c.ptr, unsafe.Pointer(&schemaBlob[0]), (C.int64_t)(len(schemaBlob))) + return ConsumeCStatusIntoError(&status) +} + // Release releases the underlying collection func (c *CCollection) Release() { C.DeleteCollection(c.ptr) diff --git a/internal/util/wrappers/qn_wrapper.go b/internal/util/wrappers/qn_wrapper.go index 9434afe39e..a2ea3ba597 100644 --- a/internal/util/wrappers/qn_wrapper.go +++ b/internal/util/wrappers/qn_wrapper.go @@ -152,6 +152,10 @@ func (qn *qnServerWrapper) DeleteBatch(ctx context.Context, in *querypb.DeleteBa return qn.QueryNode.DeleteBatch(ctx, in) } +func (qn *qnServerWrapper) UpdateSchema(ctx context.Context, in *querypb.UpdateSchemaRequest, _ ...grpc.CallOption) (*commonpb.Status, error) { + return qn.QueryNode.UpdateSchema(ctx, in) +} + func WrapQueryNodeServerAsClient(qn types.QueryNode) types.QueryNodeClient { return &qnServerWrapper{ QueryNode: qn, diff --git a/pkg/proto/messages.proto b/pkg/proto/messages.proto index 5a40768c22..a5c6807efe 100644 --- a/pkg/proto/messages.proto +++ b/pkg/proto/messages.proto @@ -4,6 +4,8 @@ package milvus.proto.messages; option go_package = "github.com/milvus-io/milvus/pkg/v2/proto/messagespb"; +import "schema.proto"; + // MessageID is the unique identifier of a message. message MessageID { string id = 1; @@ -36,6 +38,7 @@ enum MessageType { ManualFlush = 9; CreateSegment = 10; Import = 11; + SchemaChange = 12; // begin transaction message is only used for transaction, once a begin // transaction message is received, all messages combined with the // transaction message cannot be consumed until a CommitTxn message @@ -208,6 +211,17 @@ message TxnMessageHeader {} message ImportMessageHeader {} +// SchemaChangeMessageHeader is the header of CollectionSchema update message. +message SchemaChangeMessageHeader{ + int64 collection_id = 1; +} + +// SchemaChangeMessageBody is the body of CollectionSchema update message. +message SchemaChangeMessageBody{ + schema.CollectionSchema schema = 1; + uint64 modify_ts = 2; +} + /// /// Message Extra Response /// Used to add extra information when response to the client. diff --git a/pkg/proto/messagespb/messages.pb.go b/pkg/proto/messagespb/messages.pb.go index 34e529af5d..3c5fd5b4fe 100644 --- a/pkg/proto/messagespb/messages.pb.go +++ b/pkg/proto/messagespb/messages.pb.go @@ -7,6 +7,7 @@ package messagespb import ( + schemapb "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" reflect "reflect" @@ -36,6 +37,7 @@ const ( MessageType_ManualFlush MessageType = 9 MessageType_CreateSegment MessageType = 10 MessageType_Import MessageType = 11 + MessageType_SchemaChange MessageType = 12 // begin transaction message is only used for transaction, once a begin // transaction message is received, all messages combined with the // transaction message cannot be consumed until a CommitTxn message @@ -74,6 +76,7 @@ var ( 9: "ManualFlush", 10: "CreateSegment", 11: "Import", + 12: "SchemaChange", 900: "BeginTxn", 901: "CommitTxn", 902: "RollbackTxn", @@ -92,6 +95,7 @@ var ( "ManualFlush": 9, "CreateSegment": 10, "Import": 11, + "SchemaChange": 12, "BeginTxn": 900, "CommitTxn": 901, "RollbackTxn": 902, @@ -1623,6 +1627,110 @@ func (*ImportMessageHeader) Descriptor() ([]byte, []int) { return file_messages_proto_rawDescGZIP(), []int{27} } +// SchemaChangeMessageHeader is the header of CollectionSchema update message. +type SchemaChangeMessageHeader struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + CollectionId int64 `protobuf:"varint,1,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"` +} + +func (x *SchemaChangeMessageHeader) Reset() { + *x = SchemaChangeMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SchemaChangeMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SchemaChangeMessageHeader) ProtoMessage() {} + +func (x *SchemaChangeMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[28] + 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 SchemaChangeMessageHeader.ProtoReflect.Descriptor instead. +func (*SchemaChangeMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{28} +} + +func (x *SchemaChangeMessageHeader) GetCollectionId() int64 { + if x != nil { + return x.CollectionId + } + return 0 +} + +// SchemaChangeMessageBody is the body of CollectionSchema update message. +type SchemaChangeMessageBody struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Schema *schemapb.CollectionSchema `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"` + ModifyTs uint64 `protobuf:"varint,2,opt,name=modify_ts,json=modifyTs,proto3" json:"modify_ts,omitempty"` +} + +func (x *SchemaChangeMessageBody) Reset() { + *x = SchemaChangeMessageBody{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[29] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SchemaChangeMessageBody) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SchemaChangeMessageBody) ProtoMessage() {} + +func (x *SchemaChangeMessageBody) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[29] + 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 SchemaChangeMessageBody.ProtoReflect.Descriptor instead. +func (*SchemaChangeMessageBody) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{29} +} + +func (x *SchemaChangeMessageBody) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + +func (x *SchemaChangeMessageBody) GetModifyTs() uint64 { + if x != nil { + return x.ModifyTs + } + return 0 +} + // ManualFlushExtraResponse is the extra response of manual flush message. type ManualFlushExtraResponse struct { state protoimpl.MessageState @@ -1635,7 +1743,7 @@ type ManualFlushExtraResponse struct { func (x *ManualFlushExtraResponse) Reset() { *x = ManualFlushExtraResponse{} if protoimpl.UnsafeEnabled { - mi := &file_messages_proto_msgTypes[28] + mi := &file_messages_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1648,7 +1756,7 @@ func (x *ManualFlushExtraResponse) String() string { func (*ManualFlushExtraResponse) ProtoMessage() {} func (x *ManualFlushExtraResponse) ProtoReflect() protoreflect.Message { - mi := &file_messages_proto_msgTypes[28] + mi := &file_messages_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1661,7 +1769,7 @@ func (x *ManualFlushExtraResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ManualFlushExtraResponse.ProtoReflect.Descriptor instead. func (*ManualFlushExtraResponse) Descriptor() ([]byte, []int) { - return file_messages_proto_rawDescGZIP(), []int{28} + return file_messages_proto_rawDescGZIP(), []int{30} } func (x *ManualFlushExtraResponse) GetSegmentIds() []int64 { @@ -1689,7 +1797,7 @@ type TxnContext struct { func (x *TxnContext) Reset() { *x = TxnContext{} if protoimpl.UnsafeEnabled { - mi := &file_messages_proto_msgTypes[29] + mi := &file_messages_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1702,7 +1810,7 @@ func (x *TxnContext) String() string { func (*TxnContext) ProtoMessage() {} func (x *TxnContext) ProtoReflect() protoreflect.Message { - mi := &file_messages_proto_msgTypes[29] + mi := &file_messages_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1715,7 +1823,7 @@ func (x *TxnContext) ProtoReflect() protoreflect.Message { // Deprecated: Use TxnContext.ProtoReflect.Descriptor instead. func (*TxnContext) Descriptor() ([]byte, []int) { - return file_messages_proto_rawDescGZIP(), []int{29} + return file_messages_proto_rawDescGZIP(), []int{31} } func (x *TxnContext) GetTxnId() int64 { @@ -1745,7 +1853,7 @@ type RMQMessageLayout struct { func (x *RMQMessageLayout) Reset() { *x = RMQMessageLayout{} if protoimpl.UnsafeEnabled { - mi := &file_messages_proto_msgTypes[30] + mi := &file_messages_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1758,7 +1866,7 @@ func (x *RMQMessageLayout) String() string { func (*RMQMessageLayout) ProtoMessage() {} func (x *RMQMessageLayout) ProtoReflect() protoreflect.Message { - mi := &file_messages_proto_msgTypes[30] + mi := &file_messages_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1771,7 +1879,7 @@ func (x *RMQMessageLayout) ProtoReflect() protoreflect.Message { // Deprecated: Use RMQMessageLayout.ProtoReflect.Descriptor instead. func (*RMQMessageLayout) Descriptor() ([]byte, []int) { - return file_messages_proto_rawDescGZIP(), []int{30} + return file_messages_proto_rawDescGZIP(), []int{32} } func (x *RMQMessageLayout) GetPayload() []byte { @@ -1802,7 +1910,7 @@ type BroadcastHeader struct { func (x *BroadcastHeader) Reset() { *x = BroadcastHeader{} if protoimpl.UnsafeEnabled { - mi := &file_messages_proto_msgTypes[31] + mi := &file_messages_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1815,7 +1923,7 @@ func (x *BroadcastHeader) String() string { func (*BroadcastHeader) ProtoMessage() {} func (x *BroadcastHeader) ProtoReflect() protoreflect.Message { - mi := &file_messages_proto_msgTypes[31] + mi := &file_messages_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1828,7 +1936,7 @@ func (x *BroadcastHeader) ProtoReflect() protoreflect.Message { // Deprecated: Use BroadcastHeader.ProtoReflect.Descriptor instead. func (*BroadcastHeader) Descriptor() ([]byte, []int) { - return file_messages_proto_rawDescGZIP(), []int{31} + return file_messages_proto_rawDescGZIP(), []int{33} } func (x *BroadcastHeader) GetBroadcastId() uint64 { @@ -1867,7 +1975,7 @@ type ResourceKey struct { func (x *ResourceKey) Reset() { *x = ResourceKey{} if protoimpl.UnsafeEnabled { - mi := &file_messages_proto_msgTypes[32] + mi := &file_messages_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1880,7 +1988,7 @@ func (x *ResourceKey) String() string { func (*ResourceKey) ProtoMessage() {} func (x *ResourceKey) ProtoReflect() protoreflect.Message { - mi := &file_messages_proto_msgTypes[32] + mi := &file_messages_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1893,7 +2001,7 @@ func (x *ResourceKey) ProtoReflect() protoreflect.Message { // Deprecated: Use ResourceKey.ProtoReflect.Descriptor instead. func (*ResourceKey) Descriptor() ([]byte, []int) { - return file_messages_proto_rawDescGZIP(), []int{32} + return file_messages_proto_rawDescGZIP(), []int{34} } func (x *ResourceKey) GetDomain() ResourceDomain { @@ -1924,7 +2032,7 @@ type CipherHeader struct { func (x *CipherHeader) Reset() { *x = CipherHeader{} if protoimpl.UnsafeEnabled { - mi := &file_messages_proto_msgTypes[33] + mi := &file_messages_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1937,7 +2045,7 @@ func (x *CipherHeader) String() string { func (*CipherHeader) ProtoMessage() {} func (x *CipherHeader) ProtoReflect() protoreflect.Message { - mi := &file_messages_proto_msgTypes[33] + mi := &file_messages_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1950,7 +2058,7 @@ func (x *CipherHeader) ProtoReflect() protoreflect.Message { // Deprecated: Use CipherHeader.ProtoReflect.Descriptor instead. func (*CipherHeader) Descriptor() ([]byte, []int) { - return file_messages_proto_rawDescGZIP(), []int{33} + return file_messages_proto_rawDescGZIP(), []int{35} } func (x *CipherHeader) GetEzId() int64 { @@ -1979,223 +2087,237 @@ var File_messages_proto protoreflect.FileDescriptor var file_messages_proto_rawDesc = []byte{ 0x0a, 0x0e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x22, 0x1b, 0x0a, 0x09, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x49, 0x44, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x02, 0x69, 0x64, 0x22, 0xb2, 0x01, 0x0a, 0x07, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4e, 0x0a, 0x0a, 0x70, 0x72, - 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x50, - 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, - 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x72, - 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, - 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, - 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xf6, 0x01, 0x0a, 0x10, 0x49, 0x6d, - 0x6d, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x30, - 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x52, 0x02, 0x69, 0x64, - 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72, - 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x49, 0x6d, 0x6d, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, - 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, - 0x69, 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, - 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, - 0x38, 0x01, 0x22, 0x56, 0x0a, 0x10, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, - 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x18, 0x0a, 0x16, 0x4d, 0x61, - 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x42, 0x6f, 0x64, 0x79, 0x22, 0x85, 0x01, 0x0a, 0x18, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, - 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x44, 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x1a, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x1b, 0x0a, 0x09, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x49, 0x44, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, + 0x69, 0x64, 0x22, 0xb2, 0x01, 0x0a, 0x07, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x18, + 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4e, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70, + 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x50, 0x72, 0x6f, + 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x70, 0x72, + 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x70, + 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, + 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xf6, 0x01, 0x0a, 0x10, 0x49, 0x6d, 0x6d, 0x75, + 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x02, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, - 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, - 0x66, 0x6f, 0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x7e, 0x0a, 0x11, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, - 0x6f, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, - 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, - 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x49, 0x64, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x76, - 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, - 0x6f, 0x72, 0x61, 0x67, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x15, 0x0a, 0x13, - 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, - 0x6f, 0x64, 0x79, 0x22, 0x16, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x22, 0x18, 0x0a, 0x16, 0x52, - 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x42, 0x6f, 0x64, 0x79, 0x22, 0x4c, 0x0a, 0x0e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x3a, 0x0a, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x73, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x73, 0x22, 0x17, 0x0a, 0x15, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x4d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x8d, 0x01, 0x0a, - 0x13, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, - 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x44, 0x52, 0x02, 0x69, 0x64, 0x12, 0x18, + 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70, + 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x73, 0x2e, 0x49, 0x6d, 0x6d, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, + 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0x56, 0x0a, 0x10, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x42, 0x6f, 0x64, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x51, 0x0a, 0x0a, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x73, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x18, 0x0a, 0x16, 0x4d, 0x61, 0x6e, 0x75, + 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, + 0x64, 0x79, 0x22, 0x85, 0x01, 0x0a, 0x18, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, + 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x44, 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x7e, 0x0a, 0x11, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, + 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x64, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x5f, 0x76, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x74, 0x6f, 0x72, + 0x61, 0x67, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x15, 0x0a, 0x13, 0x42, 0x65, + 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, + 0x79, 0x22, 0x16, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, 0x6f, 0x64, 0x79, 0x22, 0x18, 0x0a, 0x16, 0x52, 0x6f, 0x6c, + 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x42, + 0x6f, 0x64, 0x79, 0x22, 0x4c, 0x0a, 0x0e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x3a, 0x0a, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x08, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x73, 0x22, 0x17, 0x0a, 0x15, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x8d, 0x01, 0x0a, 0x13, 0x49, + 0x6e, 0x73, 0x65, 0x72, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x51, 0x0a, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0a, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xcd, 0x01, 0x0a, 0x1a, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, + 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x72, 0x6f, 0x77, 0x73, + 0x12, 0x1f, 0x0a, 0x0b, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x53, 0x69, 0x7a, + 0x65, 0x12, 0x57, 0x0a, 0x12, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x73, 0x73, + 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, - 0x52, 0x0a, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xcd, 0x01, 0x0a, - 0x1a, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x70, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x12, - 0x0a, 0x04, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x72, 0x6f, - 0x77, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x5f, 0x73, 0x69, 0x7a, - 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x62, 0x69, 0x6e, 0x61, 0x72, 0x79, 0x53, - 0x69, 0x7a, 0x65, 0x12, 0x57, 0x0a, 0x12, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x61, - 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, - 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x11, 0x73, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x22, 0x32, 0x0a, 0x11, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, - 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, - 0x22, 0x3a, 0x0a, 0x13, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, - 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x14, 0x0a, 0x12, - 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x22, 0x1c, 0x0a, 0x1a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, - 0x22, 0x5a, 0x0a, 0x18, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, - 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, - 0x64, 0x12, 0x19, 0x0a, 0x08, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x5f, 0x74, 0x73, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x04, 0x52, 0x07, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x54, 0x73, 0x22, 0x69, 0x0a, 0x1d, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, - 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x73, 0x22, 0x42, 0x0a, 0x1b, 0x44, 0x72, 0x6f, 0x70, 0x43, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x66, 0x0a, 0x1c, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, + 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, + 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x11, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x22, 0x32, 0x0a, 0x11, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, + 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x3a, + 0x0a, 0x13, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x14, 0x0a, 0x12, 0x46, 0x6c, + 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x22, 0x1c, 0x0a, 0x1a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x5a, + 0x0a, 0x18, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, + 0x19, 0x0a, 0x08, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x5f, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x07, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x54, 0x73, 0x22, 0x69, 0x0a, 0x1d, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, - 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x49, 0x64, 0x22, 0x64, 0x0a, 0x1a, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, - 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x4e, 0x0a, 0x15, 0x42, 0x65, 0x67, - 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x12, 0x35, 0x0a, 0x16, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, - 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x15, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6c, - 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0x18, 0x0a, 0x16, 0x43, 0x6f, 0x6d, - 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x22, 0x1a, 0x0a, 0x18, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, - 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, - 0x12, 0x0a, 0x10, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x22, 0x15, 0x0a, 0x13, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x3b, 0x0a, 0x18, 0x4d, 0x61, - 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, - 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, 0x5a, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x43, 0x6f, - 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x74, 0x78, 0x6e, 0x5f, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x74, 0x78, 0x6e, 0x49, 0x64, 0x12, 0x35, 0x0a, 0x16, - 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, - 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x6b, 0x65, - 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, - 0x6e, 0x64, 0x73, 0x22, 0xc4, 0x01, 0x0a, 0x10, 0x52, 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, - 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, - 0x61, 0x64, 0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, - 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x2e, - 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, - 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, - 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, - 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9b, 0x01, 0x0a, 0x0f, 0x42, - 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x21, - 0x0a, 0x0c, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x49, - 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, - 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, - 0x47, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, - 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, - 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0c, 0x52, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, 0x5e, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x3d, 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, - 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, - 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x52, 0x06, - 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22, 0x63, 0x0a, 0x0c, 0x43, 0x69, 0x70, 0x68, - 0x65, 0x72, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x13, 0x0a, 0x05, 0x65, 0x7a, 0x5f, 0x69, - 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x65, 0x7a, 0x49, 0x64, 0x12, 0x19, 0x0a, - 0x08, 0x73, 0x61, 0x66, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, - 0x07, 0x73, 0x61, 0x66, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x61, 0x79, 0x6c, - 0x6f, 0x61, 0x64, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x0c, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x42, 0x79, 0x74, 0x65, 0x73, 0x2a, 0x88, 0x02, - 0x0a, 0x0b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, - 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x69, - 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6e, 0x73, 0x65, - 0x72, 0x74, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x03, - 0x12, 0x09, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, - 0x05, 0x12, 0x12, 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x72, - 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x12, 0x0f, 0x0a, - 0x0b, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x09, 0x12, 0x11, - 0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x10, - 0x0a, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x10, 0x0b, 0x12, 0x0d, 0x0a, - 0x08, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x10, 0x84, 0x07, 0x12, 0x0e, 0x0a, 0x09, - 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x10, 0x85, 0x07, 0x12, 0x10, 0x0a, 0x0b, - 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x10, 0x86, 0x07, 0x12, 0x08, - 0x0a, 0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7, 0x07, 0x2a, 0x82, 0x01, 0x0a, 0x08, 0x54, 0x78, 0x6e, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, - 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x42, 0x65, 0x67, 0x69, - 0x6e, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69, 0x67, - 0x68, 0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f, 0x6d, - 0x6d, 0x69, 0x74, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, - 0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f, 0x6e, - 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, - 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0x06, 0x2a, 0x6c, 0x0a, - 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, - 0x19, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, - 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x52, 0x65, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x49, 0x6d, 0x70, 0x6f, - 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x49, 0x44, 0x10, 0x01, 0x12, 0x20, 0x0a, 0x1c, 0x52, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x02, 0x42, 0x35, 0x5a, 0x33, 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, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, - 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x12, 0x23, 0x0a, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x64, 0x73, 0x22, 0x42, 0x0a, 0x1b, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x66, 0x0a, 0x1c, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, + 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x64, 0x22, 0x64, 0x0a, 0x1a, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, + 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x4e, 0x0a, 0x15, 0x42, 0x65, 0x67, 0x69, 0x6e, + 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x12, 0x35, 0x0a, 0x16, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, 0x6d, 0x69, + 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x15, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6c, 0x6c, 0x69, + 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0x18, 0x0a, 0x16, 0x43, 0x6f, 0x6d, 0x6d, 0x69, + 0x74, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x22, 0x1a, 0x0a, 0x18, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x12, 0x0a, + 0x10, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x22, 0x15, 0x0a, 0x13, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x40, 0x0a, 0x19, 0x53, 0x63, 0x68, 0x65, + 0x6d, 0x61, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x75, 0x0a, 0x17, 0x53, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x42, 0x6f, 0x64, 0x79, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, + 0x68, 0x65, 0x6d, 0x61, 0x12, 0x1b, 0x0a, 0x09, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x5f, 0x74, + 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x54, + 0x73, 0x22, 0x3b, 0x0a, 0x18, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, + 0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, + 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, 0x5a, + 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x15, 0x0a, 0x06, + 0x74, 0x78, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x74, 0x78, + 0x6e, 0x49, 0x64, 0x12, 0x35, 0x0a, 0x16, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, + 0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x15, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, + 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0xc4, 0x01, 0x0a, 0x10, 0x52, + 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x12, + 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72, 0x6f, + 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, + 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x9b, 0x01, 0x0a, 0x0f, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x48, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, + 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, 0x6f, + 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x47, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, + 0x79, 0x52, 0x0c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, + 0x5e, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x3d, + 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, + 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x52, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22, + 0x63, 0x0a, 0x0c, 0x43, 0x69, 0x70, 0x68, 0x65, 0x72, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, + 0x13, 0x0a, 0x05, 0x65, 0x7a, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, + 0x65, 0x7a, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x61, 0x66, 0x65, 0x5f, 0x6b, 0x65, 0x79, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x73, 0x61, 0x66, 0x65, 0x4b, 0x65, 0x79, 0x12, + 0x23, 0x0a, 0x0d, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x42, + 0x79, 0x74, 0x65, 0x73, 0x2a, 0x9a, 0x02, 0x0a, 0x0b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, + 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x10, 0x01, 0x12, + 0x0a, 0x0a, 0x06, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, + 0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x12, 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70, + 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, + 0x07, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x10, 0x08, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, + 0x75, 0x73, 0x68, 0x10, 0x09, 0x12, 0x11, 0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x10, 0x0a, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6d, 0x70, 0x6f, + 0x72, 0x74, 0x10, 0x0b, 0x12, 0x10, 0x0a, 0x0c, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x43, 0x68, + 0x61, 0x6e, 0x67, 0x65, 0x10, 0x0c, 0x12, 0x0d, 0x0a, 0x08, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, + 0x78, 0x6e, 0x10, 0x84, 0x07, 0x12, 0x0e, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, + 0x78, 0x6e, 0x10, 0x85, 0x07, 0x12, 0x10, 0x0a, 0x0b, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, + 0x6b, 0x54, 0x78, 0x6e, 0x10, 0x86, 0x07, 0x12, 0x08, 0x0a, 0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7, + 0x07, 0x2a, 0x82, 0x01, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, + 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, + 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, + 0x54, 0x78, 0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a, + 0x0b, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x10, 0x03, 0x12, 0x10, + 0x0a, 0x0c, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x04, + 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, + 0x6b, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, + 0x63, 0x6b, 0x65, 0x64, 0x10, 0x06, 0x2a, 0x6c, 0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x19, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, + 0x6e, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, + 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x49, 0x44, + 0x10, 0x01, 0x12, 0x20, 0x0a, 0x1c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, + 0x6d, 0x61, 0x69, 0x6e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, + 0x6d, 0x65, 0x10, 0x02, 0x42, 0x35, 0x5a, 0x33, 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, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, } var ( @@ -2211,7 +2333,7 @@ func file_messages_proto_rawDescGZIP() []byte { } var file_messages_proto_enumTypes = make([]protoimpl.EnumInfo, 3) -var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 37) +var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 39) var file_messages_proto_goTypes = []interface{}{ (MessageType)(0), // 0: milvus.proto.messages.MessageType (TxnState)(0), // 1: milvus.proto.messages.TxnState @@ -2244,32 +2366,36 @@ var file_messages_proto_goTypes = []interface{}{ (*RollbackTxnMessageHeader)(nil), // 28: milvus.proto.messages.RollbackTxnMessageHeader (*TxnMessageHeader)(nil), // 29: milvus.proto.messages.TxnMessageHeader (*ImportMessageHeader)(nil), // 30: milvus.proto.messages.ImportMessageHeader - (*ManualFlushExtraResponse)(nil), // 31: milvus.proto.messages.ManualFlushExtraResponse - (*TxnContext)(nil), // 32: milvus.proto.messages.TxnContext - (*RMQMessageLayout)(nil), // 33: milvus.proto.messages.RMQMessageLayout - (*BroadcastHeader)(nil), // 34: milvus.proto.messages.BroadcastHeader - (*ResourceKey)(nil), // 35: milvus.proto.messages.ResourceKey - (*CipherHeader)(nil), // 36: milvus.proto.messages.CipherHeader - nil, // 37: milvus.proto.messages.Message.PropertiesEntry - nil, // 38: milvus.proto.messages.ImmutableMessage.PropertiesEntry - nil, // 39: milvus.proto.messages.RMQMessageLayout.PropertiesEntry + (*SchemaChangeMessageHeader)(nil), // 31: milvus.proto.messages.SchemaChangeMessageHeader + (*SchemaChangeMessageBody)(nil), // 32: milvus.proto.messages.SchemaChangeMessageBody + (*ManualFlushExtraResponse)(nil), // 33: milvus.proto.messages.ManualFlushExtraResponse + (*TxnContext)(nil), // 34: milvus.proto.messages.TxnContext + (*RMQMessageLayout)(nil), // 35: milvus.proto.messages.RMQMessageLayout + (*BroadcastHeader)(nil), // 36: milvus.proto.messages.BroadcastHeader + (*ResourceKey)(nil), // 37: milvus.proto.messages.ResourceKey + (*CipherHeader)(nil), // 38: milvus.proto.messages.CipherHeader + nil, // 39: milvus.proto.messages.Message.PropertiesEntry + nil, // 40: milvus.proto.messages.ImmutableMessage.PropertiesEntry + nil, // 41: milvus.proto.messages.RMQMessageLayout.PropertiesEntry + (*schemapb.CollectionSchema)(nil), // 42: milvus.proto.schema.CollectionSchema } var file_messages_proto_depIdxs = []int32{ - 37, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry + 39, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry 3, // 1: milvus.proto.messages.ImmutableMessage.id:type_name -> milvus.proto.messages.MessageID - 38, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry + 40, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry 9, // 3: milvus.proto.messages.CreateSegmentMessageBody.segments:type_name -> milvus.proto.messages.CreateSegmentInfo 4, // 4: milvus.proto.messages.TxnMessageBody.messages:type_name -> milvus.proto.messages.Message 16, // 5: milvus.proto.messages.InsertMessageHeader.partitions:type_name -> milvus.proto.messages.PartitionSegmentAssignment 17, // 6: milvus.proto.messages.PartitionSegmentAssignment.segment_assignment:type_name -> milvus.proto.messages.SegmentAssignment - 39, // 7: milvus.proto.messages.RMQMessageLayout.properties:type_name -> milvus.proto.messages.RMQMessageLayout.PropertiesEntry - 35, // 8: milvus.proto.messages.BroadcastHeader.Resource_keys:type_name -> milvus.proto.messages.ResourceKey - 2, // 9: milvus.proto.messages.ResourceKey.domain:type_name -> milvus.proto.messages.ResourceDomain - 10, // [10:10] is the sub-list for method output_type - 10, // [10:10] is the sub-list for method input_type - 10, // [10:10] is the sub-list for extension type_name - 10, // [10:10] is the sub-list for extension extendee - 0, // [0:10] is the sub-list for field type_name + 42, // 7: milvus.proto.messages.SchemaChangeMessageBody.schema:type_name -> milvus.proto.schema.CollectionSchema + 41, // 8: milvus.proto.messages.RMQMessageLayout.properties:type_name -> milvus.proto.messages.RMQMessageLayout.PropertiesEntry + 37, // 9: milvus.proto.messages.BroadcastHeader.Resource_keys:type_name -> milvus.proto.messages.ResourceKey + 2, // 10: milvus.proto.messages.ResourceKey.domain:type_name -> milvus.proto.messages.ResourceDomain + 11, // [11:11] is the sub-list for method output_type + 11, // [11:11] is the sub-list for method input_type + 11, // [11:11] is the sub-list for extension type_name + 11, // [11:11] is the sub-list for extension extendee + 0, // [0:11] is the sub-list for field type_name } func init() { file_messages_proto_init() } @@ -2615,7 +2741,7 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ManualFlushExtraResponse); i { + switch v := v.(*SchemaChangeMessageHeader); i { case 0: return &v.state case 1: @@ -2627,7 +2753,7 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TxnContext); i { + switch v := v.(*SchemaChangeMessageBody); i { case 0: return &v.state case 1: @@ -2639,7 +2765,7 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RMQMessageLayout); i { + switch v := v.(*ManualFlushExtraResponse); i { case 0: return &v.state case 1: @@ -2651,7 +2777,7 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BroadcastHeader); i { + switch v := v.(*TxnContext); i { case 0: return &v.state case 1: @@ -2663,7 +2789,7 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResourceKey); i { + switch v := v.(*RMQMessageLayout); i { case 0: return &v.state case 1: @@ -2675,6 +2801,30 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BroadcastHeader); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ResourceKey); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*CipherHeader); i { case 0: return &v.state @@ -2693,7 +2843,7 @@ func file_messages_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_messages_proto_rawDesc, NumEnums: 3, - NumMessages: 37, + NumMessages: 39, NumExtensions: 0, NumServices: 0, }, diff --git a/pkg/proto/query_coord.proto b/pkg/proto/query_coord.proto index 23bc6ce05b..50fc808321 100644 --- a/pkg/proto/query_coord.proto +++ b/pkg/proto/query_coord.proto @@ -167,6 +167,7 @@ service QueryNode { // it's basically same as `Delete` but cost less memory pressure. rpc DeleteBatch(DeleteBatchRequest) returns (DeleteBatchResponse) { } + rpc UpdateSchema(UpdateSchemaRequest) returns (common.Status) {} } // --------------------QueryCoord grpc request and response proto------------------ @@ -944,3 +945,9 @@ message UpdateLoadConfigRequest { int32 replica_number = 4; repeated string resource_groups = 5; } + +message UpdateSchemaRequest { + common.MsgBase base = 1; + int64 collectionID = 2; + schema.CollectionSchema schema = 3; +} \ No newline at end of file diff --git a/pkg/proto/querypb/query_coord.pb.go b/pkg/proto/querypb/query_coord.pb.go index d158991631..bdacaf1485 100644 --- a/pkg/proto/querypb/query_coord.pb.go +++ b/pkg/proto/querypb/query_coord.pb.go @@ -7225,6 +7225,69 @@ func (x *UpdateLoadConfigRequest) GetResourceGroups() []string { return nil } +type UpdateSchemaRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` + Schema *schemapb.CollectionSchema `protobuf:"bytes,3,opt,name=schema,proto3" json:"schema,omitempty"` +} + +func (x *UpdateSchemaRequest) Reset() { + *x = UpdateSchemaRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[86] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *UpdateSchemaRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*UpdateSchemaRequest) ProtoMessage() {} + +func (x *UpdateSchemaRequest) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[86] + 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 UpdateSchemaRequest.ProtoReflect.Descriptor instead. +func (*UpdateSchemaRequest) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{86} +} + +func (x *UpdateSchemaRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *UpdateSchemaRequest) GetCollectionID() int64 { + if x != nil { + return x.CollectionID + } + return 0 +} + +func (x *UpdateSchemaRequest) GetSchema() *schemapb.CollectionSchema { + if x != nil { + return x.Schema + } + return nil +} + var File_query_coord_proto protoreflect.FileDescriptor var file_query_coord_proto_rawDesc = []byte{ @@ -8661,433 +8724,450 @@ var file_query_coord_proto_rawDesc = []byte{ 0x70, 0x6c, 0x69, 0x63, 0x61, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, - 0x6f, 0x75, 0x70, 0x73, 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, 0x87, 0x1d, 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, 0x50, + 0x6f, 0x75, 0x70, 0x73, 0x22, 0xaa, 0x01, 0x0a, 0x13, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, + 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x22, + 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, + 0x61, 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, 0x87, + 0x1d, 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, 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, - 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, + 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, 0x57, 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, 0x00, 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, 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, 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, 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, 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, 0x32, 0x8b, 0x14, 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, 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, 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, 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, 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, 0x57, 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, 0x00, 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, 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, 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, 0x32, 0xb3, 0x13, 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, + 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, 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, 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, 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, + 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, 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, 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, + 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, 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, + 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, 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, 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, + 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, 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, 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, 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, + 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, 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, 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, + 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, 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 ( @@ -9103,7 +9183,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, 104) +var file_query_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 105) var file_query_coord_proto_goTypes = []interface{}{ (LoadScope)(0), // 0: milvus.proto.query.LoadScope (DataScope)(0), // 1: milvus.proto.query.DataScope @@ -9198,360 +9278,365 @@ var file_query_coord_proto_goTypes = []interface{}{ (*TransferChannelRequest)(nil), // 90: milvus.proto.query.TransferChannelRequest (*CheckQueryNodeDistributionRequest)(nil), // 91: milvus.proto.query.CheckQueryNodeDistributionRequest (*UpdateLoadConfigRequest)(nil), // 92: milvus.proto.query.UpdateLoadConfigRequest - nil, // 93: milvus.proto.query.LoadCollectionRequest.FieldIndexIDEntry - nil, // 94: milvus.proto.query.LoadPartitionsRequest.FieldIndexIDEntry - nil, // 95: milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry - nil, // 96: milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry - nil, // 97: milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry - nil, // 98: milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry - nil, // 99: milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry - nil, // 100: milvus.proto.query.LeaderView.SegmentDistEntry - nil, // 101: milvus.proto.query.LeaderView.GrowingSegmentsEntry - nil, // 102: milvus.proto.query.LeaderView.PartitionStatsVersionsEntry - nil, // 103: milvus.proto.query.SegmentVersionInfo.IndexInfoEntry - nil, // 104: milvus.proto.query.CollectionLoadInfo.FieldIndexIDEntry - nil, // 105: milvus.proto.query.PartitionLoadInfo.FieldIndexIDEntry - nil, // 106: milvus.proto.query.Replica.ChannelNodeInfosEntry - nil, // 107: milvus.proto.query.SyncAction.PartitionStatsVersionsEntry - nil, // 108: milvus.proto.query.ResourceGroupInfo.NumLoadedReplicaEntry - nil, // 109: milvus.proto.query.ResourceGroupInfo.NumOutgoingNodeEntry - nil, // 110: milvus.proto.query.ResourceGroupInfo.NumIncomingNodeEntry - (*commonpb.MsgBase)(nil), // 111: milvus.proto.common.MsgBase - (*commonpb.Status)(nil), // 112: milvus.proto.common.Status - (*schemapb.LongArray)(nil), // 113: milvus.proto.schema.LongArray - (*schemapb.CollectionSchema)(nil), // 114: milvus.proto.schema.CollectionSchema - (*internalpb.GetStatisticsRequest)(nil), // 115: milvus.proto.internal.GetStatisticsRequest - (*indexpb.IndexInfo)(nil), // 116: milvus.proto.index.IndexInfo - (*commonpb.KeyValuePair)(nil), // 117: milvus.proto.common.KeyValuePair - (*datapb.VchannelInfo)(nil), // 118: milvus.proto.data.VchannelInfo - (*datapb.SegmentInfo)(nil), // 119: milvus.proto.data.SegmentInfo - (*datapb.FieldBinlog)(nil), // 120: milvus.proto.data.FieldBinlog - (*msgpb.MsgPosition)(nil), // 121: milvus.proto.msg.MsgPosition - (datapb.SegmentLevel)(0), // 122: milvus.proto.data.SegmentLevel - (*internalpb.SearchRequest)(nil), // 123: milvus.proto.internal.SearchRequest - (*internalpb.RetrieveRequest)(nil), // 124: milvus.proto.internal.RetrieveRequest - (commonpb.SegmentState)(0), // 125: milvus.proto.common.SegmentState - (*rgpb.ResourceGroupConfig)(nil), // 126: milvus.proto.rg.ResourceGroupConfig - (*commonpb.NodeInfo)(nil), // 127: milvus.proto.common.NodeInfo - (*schemapb.IDs)(nil), // 128: milvus.proto.schema.IDs - (*datapb.TextIndexStats)(nil), // 129: milvus.proto.data.TextIndexStats - (*datapb.JsonKeyStats)(nil), // 130: milvus.proto.data.JsonKeyStats - (*internalpb.ShowConfigurationsRequest)(nil), // 131: milvus.proto.internal.ShowConfigurationsRequest - (*milvuspb.GetMetricsRequest)(nil), // 132: milvus.proto.milvus.GetMetricsRequest - (*milvuspb.GetReplicasRequest)(nil), // 133: milvus.proto.milvus.GetReplicasRequest - (*milvuspb.CheckHealthRequest)(nil), // 134: milvus.proto.milvus.CheckHealthRequest - (*milvuspb.CreateResourceGroupRequest)(nil), // 135: milvus.proto.milvus.CreateResourceGroupRequest - (*milvuspb.DropResourceGroupRequest)(nil), // 136: milvus.proto.milvus.DropResourceGroupRequest - (*milvuspb.TransferNodeRequest)(nil), // 137: milvus.proto.milvus.TransferNodeRequest - (*milvuspb.ListResourceGroupsRequest)(nil), // 138: milvus.proto.milvus.ListResourceGroupsRequest - (*milvuspb.GetComponentStatesRequest)(nil), // 139: milvus.proto.milvus.GetComponentStatesRequest - (*internalpb.GetTimeTickChannelRequest)(nil), // 140: milvus.proto.internal.GetTimeTickChannelRequest - (*internalpb.GetStatisticsChannelRequest)(nil), // 141: milvus.proto.internal.GetStatisticsChannelRequest - (*internalpb.ShowConfigurationsResponse)(nil), // 142: milvus.proto.internal.ShowConfigurationsResponse - (*milvuspb.GetMetricsResponse)(nil), // 143: milvus.proto.milvus.GetMetricsResponse - (*milvuspb.GetReplicasResponse)(nil), // 144: milvus.proto.milvus.GetReplicasResponse - (*milvuspb.CheckHealthResponse)(nil), // 145: milvus.proto.milvus.CheckHealthResponse - (*milvuspb.ListResourceGroupsResponse)(nil), // 146: milvus.proto.milvus.ListResourceGroupsResponse - (*milvuspb.ComponentStates)(nil), // 147: milvus.proto.milvus.ComponentStates - (*milvuspb.StringResponse)(nil), // 148: milvus.proto.milvus.StringResponse - (*internalpb.GetStatisticsResponse)(nil), // 149: milvus.proto.internal.GetStatisticsResponse - (*internalpb.SearchResults)(nil), // 150: milvus.proto.internal.SearchResults - (*internalpb.RetrieveResults)(nil), // 151: milvus.proto.internal.RetrieveResults + (*UpdateSchemaRequest)(nil), // 93: milvus.proto.query.UpdateSchemaRequest + nil, // 94: milvus.proto.query.LoadCollectionRequest.FieldIndexIDEntry + nil, // 95: milvus.proto.query.LoadPartitionsRequest.FieldIndexIDEntry + nil, // 96: milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry + nil, // 97: milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry + nil, // 98: milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry + nil, // 99: milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry + nil, // 100: milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry + nil, // 101: milvus.proto.query.LeaderView.SegmentDistEntry + nil, // 102: milvus.proto.query.LeaderView.GrowingSegmentsEntry + nil, // 103: milvus.proto.query.LeaderView.PartitionStatsVersionsEntry + nil, // 104: milvus.proto.query.SegmentVersionInfo.IndexInfoEntry + nil, // 105: milvus.proto.query.CollectionLoadInfo.FieldIndexIDEntry + nil, // 106: milvus.proto.query.PartitionLoadInfo.FieldIndexIDEntry + nil, // 107: milvus.proto.query.Replica.ChannelNodeInfosEntry + nil, // 108: milvus.proto.query.SyncAction.PartitionStatsVersionsEntry + nil, // 109: milvus.proto.query.ResourceGroupInfo.NumLoadedReplicaEntry + nil, // 110: milvus.proto.query.ResourceGroupInfo.NumOutgoingNodeEntry + nil, // 111: milvus.proto.query.ResourceGroupInfo.NumIncomingNodeEntry + (*commonpb.MsgBase)(nil), // 112: milvus.proto.common.MsgBase + (*commonpb.Status)(nil), // 113: milvus.proto.common.Status + (*schemapb.LongArray)(nil), // 114: milvus.proto.schema.LongArray + (*schemapb.CollectionSchema)(nil), // 115: milvus.proto.schema.CollectionSchema + (*internalpb.GetStatisticsRequest)(nil), // 116: milvus.proto.internal.GetStatisticsRequest + (*indexpb.IndexInfo)(nil), // 117: milvus.proto.index.IndexInfo + (*commonpb.KeyValuePair)(nil), // 118: milvus.proto.common.KeyValuePair + (*datapb.VchannelInfo)(nil), // 119: milvus.proto.data.VchannelInfo + (*datapb.SegmentInfo)(nil), // 120: milvus.proto.data.SegmentInfo + (*datapb.FieldBinlog)(nil), // 121: milvus.proto.data.FieldBinlog + (*msgpb.MsgPosition)(nil), // 122: milvus.proto.msg.MsgPosition + (datapb.SegmentLevel)(0), // 123: milvus.proto.data.SegmentLevel + (*internalpb.SearchRequest)(nil), // 124: milvus.proto.internal.SearchRequest + (*internalpb.RetrieveRequest)(nil), // 125: milvus.proto.internal.RetrieveRequest + (commonpb.SegmentState)(0), // 126: milvus.proto.common.SegmentState + (*rgpb.ResourceGroupConfig)(nil), // 127: milvus.proto.rg.ResourceGroupConfig + (*commonpb.NodeInfo)(nil), // 128: milvus.proto.common.NodeInfo + (*schemapb.IDs)(nil), // 129: milvus.proto.schema.IDs + (*datapb.TextIndexStats)(nil), // 130: milvus.proto.data.TextIndexStats + (*datapb.JsonKeyStats)(nil), // 131: milvus.proto.data.JsonKeyStats + (*internalpb.ShowConfigurationsRequest)(nil), // 132: milvus.proto.internal.ShowConfigurationsRequest + (*milvuspb.GetMetricsRequest)(nil), // 133: milvus.proto.milvus.GetMetricsRequest + (*milvuspb.GetReplicasRequest)(nil), // 134: milvus.proto.milvus.GetReplicasRequest + (*milvuspb.CheckHealthRequest)(nil), // 135: milvus.proto.milvus.CheckHealthRequest + (*milvuspb.CreateResourceGroupRequest)(nil), // 136: milvus.proto.milvus.CreateResourceGroupRequest + (*milvuspb.DropResourceGroupRequest)(nil), // 137: milvus.proto.milvus.DropResourceGroupRequest + (*milvuspb.TransferNodeRequest)(nil), // 138: milvus.proto.milvus.TransferNodeRequest + (*milvuspb.ListResourceGroupsRequest)(nil), // 139: milvus.proto.milvus.ListResourceGroupsRequest + (*milvuspb.GetComponentStatesRequest)(nil), // 140: milvus.proto.milvus.GetComponentStatesRequest + (*internalpb.GetTimeTickChannelRequest)(nil), // 141: milvus.proto.internal.GetTimeTickChannelRequest + (*internalpb.GetStatisticsChannelRequest)(nil), // 142: milvus.proto.internal.GetStatisticsChannelRequest + (*internalpb.ShowConfigurationsResponse)(nil), // 143: milvus.proto.internal.ShowConfigurationsResponse + (*milvuspb.GetMetricsResponse)(nil), // 144: milvus.proto.milvus.GetMetricsResponse + (*milvuspb.GetReplicasResponse)(nil), // 145: milvus.proto.milvus.GetReplicasResponse + (*milvuspb.CheckHealthResponse)(nil), // 146: milvus.proto.milvus.CheckHealthResponse + (*milvuspb.ListResourceGroupsResponse)(nil), // 147: milvus.proto.milvus.ListResourceGroupsResponse + (*milvuspb.ComponentStates)(nil), // 148: milvus.proto.milvus.ComponentStates + (*milvuspb.StringResponse)(nil), // 149: milvus.proto.milvus.StringResponse + (*internalpb.GetStatisticsResponse)(nil), // 150: milvus.proto.internal.GetStatisticsResponse + (*internalpb.SearchResults)(nil), // 151: milvus.proto.internal.SearchResults + (*internalpb.RetrieveResults)(nil), // 152: milvus.proto.internal.RetrieveResults } var file_query_coord_proto_depIdxs = []int32{ - 111, // 0: milvus.proto.query.ShowCollectionsRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 1: milvus.proto.query.ShowCollectionsResponse.status:type_name -> milvus.proto.common.Status - 113, // 2: milvus.proto.query.ShowCollectionsResponse.load_fields:type_name -> milvus.proto.schema.LongArray - 111, // 3: milvus.proto.query.ShowPartitionsRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 4: milvus.proto.query.ShowPartitionsResponse.status:type_name -> milvus.proto.common.Status - 111, // 5: milvus.proto.query.LoadCollectionRequest.base:type_name -> milvus.proto.common.MsgBase - 114, // 6: milvus.proto.query.LoadCollectionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 93, // 7: milvus.proto.query.LoadCollectionRequest.field_indexID:type_name -> milvus.proto.query.LoadCollectionRequest.FieldIndexIDEntry - 111, // 8: milvus.proto.query.ReleaseCollectionRequest.base:type_name -> milvus.proto.common.MsgBase - 115, // 9: milvus.proto.query.GetStatisticsRequest.req:type_name -> milvus.proto.internal.GetStatisticsRequest + 112, // 0: milvus.proto.query.ShowCollectionsRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 1: milvus.proto.query.ShowCollectionsResponse.status:type_name -> milvus.proto.common.Status + 114, // 2: milvus.proto.query.ShowCollectionsResponse.load_fields:type_name -> milvus.proto.schema.LongArray + 112, // 3: milvus.proto.query.ShowPartitionsRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 4: milvus.proto.query.ShowPartitionsResponse.status:type_name -> milvus.proto.common.Status + 112, // 5: milvus.proto.query.LoadCollectionRequest.base:type_name -> milvus.proto.common.MsgBase + 115, // 6: milvus.proto.query.LoadCollectionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 94, // 7: milvus.proto.query.LoadCollectionRequest.field_indexID:type_name -> milvus.proto.query.LoadCollectionRequest.FieldIndexIDEntry + 112, // 8: milvus.proto.query.ReleaseCollectionRequest.base:type_name -> milvus.proto.common.MsgBase + 116, // 9: milvus.proto.query.GetStatisticsRequest.req:type_name -> milvus.proto.internal.GetStatisticsRequest 1, // 10: milvus.proto.query.GetStatisticsRequest.scope:type_name -> milvus.proto.query.DataScope - 111, // 11: milvus.proto.query.LoadPartitionsRequest.base:type_name -> milvus.proto.common.MsgBase - 114, // 12: milvus.proto.query.LoadPartitionsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 94, // 13: milvus.proto.query.LoadPartitionsRequest.field_indexID:type_name -> milvus.proto.query.LoadPartitionsRequest.FieldIndexIDEntry - 116, // 14: milvus.proto.query.LoadPartitionsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo - 111, // 15: milvus.proto.query.ReleasePartitionsRequest.base:type_name -> milvus.proto.common.MsgBase - 111, // 16: milvus.proto.query.GetPartitionStatesRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 17: milvus.proto.query.GetPartitionStatesResponse.status:type_name -> milvus.proto.common.Status + 112, // 11: milvus.proto.query.LoadPartitionsRequest.base:type_name -> milvus.proto.common.MsgBase + 115, // 12: milvus.proto.query.LoadPartitionsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 95, // 13: milvus.proto.query.LoadPartitionsRequest.field_indexID:type_name -> milvus.proto.query.LoadPartitionsRequest.FieldIndexIDEntry + 117, // 14: milvus.proto.query.LoadPartitionsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 112, // 15: milvus.proto.query.ReleasePartitionsRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 16: milvus.proto.query.GetPartitionStatesRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 17: milvus.proto.query.GetPartitionStatesResponse.status:type_name -> milvus.proto.common.Status 42, // 18: milvus.proto.query.GetPartitionStatesResponse.partition_descriptions:type_name -> milvus.proto.query.PartitionStates - 111, // 19: milvus.proto.query.GetSegmentInfoRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 20: milvus.proto.query.GetSegmentInfoResponse.status:type_name -> milvus.proto.common.Status + 112, // 19: milvus.proto.query.GetSegmentInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 20: milvus.proto.query.GetSegmentInfoResponse.status:type_name -> milvus.proto.common.Status 43, // 21: milvus.proto.query.GetSegmentInfoResponse.infos:type_name -> milvus.proto.query.SegmentInfo - 111, // 22: milvus.proto.query.GetShardLeadersRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 23: milvus.proto.query.GetShardLeadersResponse.status:type_name -> milvus.proto.common.Status + 112, // 22: milvus.proto.query.GetShardLeadersRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 23: milvus.proto.query.GetShardLeadersResponse.status:type_name -> milvus.proto.common.Status 23, // 24: milvus.proto.query.GetShardLeadersResponse.shards:type_name -> milvus.proto.query.ShardLeadersList - 111, // 25: milvus.proto.query.UpdateResourceGroupsRequest.base:type_name -> milvus.proto.common.MsgBase - 95, // 26: milvus.proto.query.UpdateResourceGroupsRequest.resource_groups:type_name -> milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry - 111, // 27: milvus.proto.query.SyncNewCreatedPartitionRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 25: milvus.proto.query.UpdateResourceGroupsRequest.base:type_name -> milvus.proto.common.MsgBase + 96, // 26: milvus.proto.query.UpdateResourceGroupsRequest.resource_groups:type_name -> milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry + 112, // 27: milvus.proto.query.SyncNewCreatedPartitionRequest.base:type_name -> milvus.proto.common.MsgBase 4, // 28: milvus.proto.query.LoadMetaInfo.load_type:type_name -> milvus.proto.query.LoadType - 117, // 29: milvus.proto.query.LoadMetaInfo.db_properties:type_name -> milvus.proto.common.KeyValuePair - 111, // 30: milvus.proto.query.WatchDmChannelsRequest.base:type_name -> milvus.proto.common.MsgBase - 118, // 31: milvus.proto.query.WatchDmChannelsRequest.infos:type_name -> milvus.proto.data.VchannelInfo - 114, // 32: milvus.proto.query.WatchDmChannelsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 119, // 33: milvus.proto.query.WatchDmChannelsRequest.exclude_infos:type_name -> milvus.proto.data.SegmentInfo + 118, // 29: milvus.proto.query.LoadMetaInfo.db_properties:type_name -> milvus.proto.common.KeyValuePair + 112, // 30: milvus.proto.query.WatchDmChannelsRequest.base:type_name -> milvus.proto.common.MsgBase + 119, // 31: milvus.proto.query.WatchDmChannelsRequest.infos:type_name -> milvus.proto.data.VchannelInfo + 115, // 32: milvus.proto.query.WatchDmChannelsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 120, // 33: milvus.proto.query.WatchDmChannelsRequest.exclude_infos:type_name -> milvus.proto.data.SegmentInfo 25, // 34: milvus.proto.query.WatchDmChannelsRequest.load_meta:type_name -> milvus.proto.query.LoadMetaInfo - 96, // 35: milvus.proto.query.WatchDmChannelsRequest.segment_infos:type_name -> milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry - 116, // 36: milvus.proto.query.WatchDmChannelsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo - 111, // 37: milvus.proto.query.UnsubDmChannelRequest.base:type_name -> milvus.proto.common.MsgBase - 120, // 38: milvus.proto.query.SegmentLoadInfo.binlog_paths:type_name -> milvus.proto.data.FieldBinlog - 120, // 39: milvus.proto.query.SegmentLoadInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog - 120, // 40: milvus.proto.query.SegmentLoadInfo.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 97, // 35: milvus.proto.query.WatchDmChannelsRequest.segment_infos:type_name -> milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry + 117, // 36: milvus.proto.query.WatchDmChannelsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 112, // 37: milvus.proto.query.UnsubDmChannelRequest.base:type_name -> milvus.proto.common.MsgBase + 121, // 38: milvus.proto.query.SegmentLoadInfo.binlog_paths:type_name -> milvus.proto.data.FieldBinlog + 121, // 39: milvus.proto.query.SegmentLoadInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog + 121, // 40: milvus.proto.query.SegmentLoadInfo.deltalogs:type_name -> milvus.proto.data.FieldBinlog 29, // 41: milvus.proto.query.SegmentLoadInfo.index_infos:type_name -> milvus.proto.query.FieldIndexInfo - 121, // 42: milvus.proto.query.SegmentLoadInfo.start_position:type_name -> milvus.proto.msg.MsgPosition - 121, // 43: milvus.proto.query.SegmentLoadInfo.delta_position:type_name -> milvus.proto.msg.MsgPosition - 122, // 44: milvus.proto.query.SegmentLoadInfo.level:type_name -> milvus.proto.data.SegmentLevel - 97, // 45: milvus.proto.query.SegmentLoadInfo.textStatsLogs:type_name -> milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry - 120, // 46: milvus.proto.query.SegmentLoadInfo.bm25logs:type_name -> milvus.proto.data.FieldBinlog - 98, // 47: milvus.proto.query.SegmentLoadInfo.jsonKeyStatsLogs:type_name -> milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry - 117, // 48: milvus.proto.query.FieldIndexInfo.index_params:type_name -> milvus.proto.common.KeyValuePair - 111, // 49: milvus.proto.query.LoadSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 122, // 42: milvus.proto.query.SegmentLoadInfo.start_position:type_name -> milvus.proto.msg.MsgPosition + 122, // 43: milvus.proto.query.SegmentLoadInfo.delta_position:type_name -> milvus.proto.msg.MsgPosition + 123, // 44: milvus.proto.query.SegmentLoadInfo.level:type_name -> milvus.proto.data.SegmentLevel + 98, // 45: milvus.proto.query.SegmentLoadInfo.textStatsLogs:type_name -> milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry + 121, // 46: milvus.proto.query.SegmentLoadInfo.bm25logs:type_name -> milvus.proto.data.FieldBinlog + 99, // 47: milvus.proto.query.SegmentLoadInfo.jsonKeyStatsLogs:type_name -> milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry + 118, // 48: milvus.proto.query.FieldIndexInfo.index_params:type_name -> milvus.proto.common.KeyValuePair + 112, // 49: milvus.proto.query.LoadSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase 28, // 50: milvus.proto.query.LoadSegmentsRequest.infos:type_name -> milvus.proto.query.SegmentLoadInfo - 114, // 51: milvus.proto.query.LoadSegmentsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 115, // 51: milvus.proto.query.LoadSegmentsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema 25, // 52: milvus.proto.query.LoadSegmentsRequest.load_meta:type_name -> milvus.proto.query.LoadMetaInfo - 121, // 53: milvus.proto.query.LoadSegmentsRequest.delta_positions:type_name -> milvus.proto.msg.MsgPosition + 122, // 53: milvus.proto.query.LoadSegmentsRequest.delta_positions:type_name -> milvus.proto.msg.MsgPosition 0, // 54: milvus.proto.query.LoadSegmentsRequest.load_scope:type_name -> milvus.proto.query.LoadScope - 116, // 55: milvus.proto.query.LoadSegmentsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo - 111, // 56: milvus.proto.query.ReleaseSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 117, // 55: milvus.proto.query.LoadSegmentsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 112, // 56: milvus.proto.query.ReleaseSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase 1, // 57: milvus.proto.query.ReleaseSegmentsRequest.scope:type_name -> milvus.proto.query.DataScope - 121, // 58: milvus.proto.query.ReleaseSegmentsRequest.checkpoint:type_name -> milvus.proto.msg.MsgPosition - 123, // 59: milvus.proto.query.SearchRequest.req:type_name -> milvus.proto.internal.SearchRequest + 122, // 58: milvus.proto.query.ReleaseSegmentsRequest.checkpoint:type_name -> milvus.proto.msg.MsgPosition + 124, // 59: milvus.proto.query.SearchRequest.req:type_name -> milvus.proto.internal.SearchRequest 1, // 60: milvus.proto.query.SearchRequest.scope:type_name -> milvus.proto.query.DataScope - 124, // 61: milvus.proto.query.QueryRequest.req:type_name -> milvus.proto.internal.RetrieveRequest + 125, // 61: milvus.proto.query.QueryRequest.req:type_name -> milvus.proto.internal.RetrieveRequest 1, // 62: milvus.proto.query.QueryRequest.scope:type_name -> milvus.proto.query.DataScope - 111, // 63: milvus.proto.query.SyncReplicaSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 63: milvus.proto.query.SyncReplicaSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase 35, // 64: milvus.proto.query.SyncReplicaSegmentsRequest.replica_segments:type_name -> milvus.proto.query.ReplicaSegmentsInfo - 111, // 65: milvus.proto.query.GetLoadInfoRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 66: milvus.proto.query.GetLoadInfoResponse.status:type_name -> milvus.proto.common.Status - 114, // 67: milvus.proto.query.GetLoadInfoResponse.schema:type_name -> milvus.proto.schema.CollectionSchema + 112, // 65: milvus.proto.query.GetLoadInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 66: milvus.proto.query.GetLoadInfoResponse.status:type_name -> milvus.proto.common.Status + 115, // 67: milvus.proto.query.GetLoadInfoResponse.schema:type_name -> milvus.proto.schema.CollectionSchema 4, // 68: milvus.proto.query.GetLoadInfoResponse.load_type:type_name -> milvus.proto.query.LoadType - 111, // 69: milvus.proto.query.HandoffSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 69: milvus.proto.query.HandoffSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase 43, // 70: milvus.proto.query.HandoffSegmentsRequest.segmentInfos:type_name -> milvus.proto.query.SegmentInfo - 111, // 71: milvus.proto.query.LoadBalanceRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 71: milvus.proto.query.LoadBalanceRequest.base:type_name -> milvus.proto.common.MsgBase 3, // 72: milvus.proto.query.LoadBalanceRequest.balance_reason:type_name -> milvus.proto.query.TriggerCondition 43, // 73: milvus.proto.query.QueryChannelInfo.global_sealed_segments:type_name -> milvus.proto.query.SegmentInfo - 121, // 74: milvus.proto.query.QueryChannelInfo.seek_position:type_name -> milvus.proto.msg.MsgPosition + 122, // 74: milvus.proto.query.QueryChannelInfo.seek_position:type_name -> milvus.proto.msg.MsgPosition 2, // 75: milvus.proto.query.PartitionStates.state:type_name -> milvus.proto.query.PartitionState - 125, // 76: milvus.proto.query.SegmentInfo.segment_state:type_name -> milvus.proto.common.SegmentState + 126, // 76: milvus.proto.query.SegmentInfo.segment_state:type_name -> milvus.proto.common.SegmentState 29, // 77: milvus.proto.query.SegmentInfo.index_infos:type_name -> milvus.proto.query.FieldIndexInfo - 122, // 78: milvus.proto.query.SegmentInfo.level:type_name -> milvus.proto.data.SegmentLevel + 123, // 78: milvus.proto.query.SegmentInfo.level:type_name -> milvus.proto.data.SegmentLevel 42, // 79: milvus.proto.query.CollectionInfo.partition_states:type_name -> milvus.proto.query.PartitionStates 4, // 80: milvus.proto.query.CollectionInfo.load_type:type_name -> milvus.proto.query.LoadType - 114, // 81: milvus.proto.query.CollectionInfo.schema:type_name -> milvus.proto.schema.CollectionSchema + 115, // 81: milvus.proto.query.CollectionInfo.schema:type_name -> milvus.proto.schema.CollectionSchema 45, // 82: milvus.proto.query.UnsubscribeChannelInfo.collection_channels:type_name -> milvus.proto.query.UnsubscribeChannels 43, // 83: milvus.proto.query.SegmentChangeInfo.online_segments:type_name -> milvus.proto.query.SegmentInfo 43, // 84: milvus.proto.query.SegmentChangeInfo.offline_segments:type_name -> milvus.proto.query.SegmentInfo - 111, // 85: milvus.proto.query.SealedSegmentsChangeInfo.base:type_name -> milvus.proto.common.MsgBase + 112, // 85: milvus.proto.query.SealedSegmentsChangeInfo.base:type_name -> milvus.proto.common.MsgBase 47, // 86: milvus.proto.query.SealedSegmentsChangeInfo.infos:type_name -> milvus.proto.query.SegmentChangeInfo - 111, // 87: milvus.proto.query.GetDataDistributionRequest.base:type_name -> milvus.proto.common.MsgBase - 99, // 88: milvus.proto.query.GetDataDistributionRequest.checkpoints:type_name -> milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry - 112, // 89: milvus.proto.query.GetDataDistributionResponse.status:type_name -> milvus.proto.common.Status + 112, // 87: milvus.proto.query.GetDataDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 100, // 88: milvus.proto.query.GetDataDistributionRequest.checkpoints:type_name -> milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry + 113, // 89: milvus.proto.query.GetDataDistributionResponse.status:type_name -> milvus.proto.common.Status 53, // 90: milvus.proto.query.GetDataDistributionResponse.segments:type_name -> milvus.proto.query.SegmentVersionInfo 54, // 91: milvus.proto.query.GetDataDistributionResponse.channels:type_name -> milvus.proto.query.ChannelVersionInfo 51, // 92: milvus.proto.query.GetDataDistributionResponse.leader_views:type_name -> milvus.proto.query.LeaderView - 100, // 93: milvus.proto.query.LeaderView.segment_dist:type_name -> milvus.proto.query.LeaderView.SegmentDistEntry - 101, // 94: milvus.proto.query.LeaderView.growing_segments:type_name -> milvus.proto.query.LeaderView.GrowingSegmentsEntry - 102, // 95: milvus.proto.query.LeaderView.partition_stats_versions:type_name -> milvus.proto.query.LeaderView.PartitionStatsVersionsEntry - 103, // 96: milvus.proto.query.SegmentVersionInfo.index_info:type_name -> milvus.proto.query.SegmentVersionInfo.IndexInfoEntry - 122, // 97: milvus.proto.query.SegmentVersionInfo.level:type_name -> milvus.proto.data.SegmentLevel + 101, // 93: milvus.proto.query.LeaderView.segment_dist:type_name -> milvus.proto.query.LeaderView.SegmentDistEntry + 102, // 94: milvus.proto.query.LeaderView.growing_segments:type_name -> milvus.proto.query.LeaderView.GrowingSegmentsEntry + 103, // 95: milvus.proto.query.LeaderView.partition_stats_versions:type_name -> milvus.proto.query.LeaderView.PartitionStatsVersionsEntry + 104, // 96: milvus.proto.query.SegmentVersionInfo.index_info:type_name -> milvus.proto.query.SegmentVersionInfo.IndexInfoEntry + 123, // 97: milvus.proto.query.SegmentVersionInfo.level:type_name -> milvus.proto.data.SegmentLevel 5, // 98: milvus.proto.query.CollectionLoadInfo.status:type_name -> milvus.proto.query.LoadStatus - 104, // 99: milvus.proto.query.CollectionLoadInfo.field_indexID:type_name -> milvus.proto.query.CollectionLoadInfo.FieldIndexIDEntry + 105, // 99: milvus.proto.query.CollectionLoadInfo.field_indexID:type_name -> milvus.proto.query.CollectionLoadInfo.FieldIndexIDEntry 4, // 100: milvus.proto.query.CollectionLoadInfo.load_type:type_name -> milvus.proto.query.LoadType 5, // 101: milvus.proto.query.PartitionLoadInfo.status:type_name -> milvus.proto.query.LoadStatus - 105, // 102: milvus.proto.query.PartitionLoadInfo.field_indexID:type_name -> milvus.proto.query.PartitionLoadInfo.FieldIndexIDEntry - 106, // 103: milvus.proto.query.Replica.channel_node_infos:type_name -> milvus.proto.query.Replica.ChannelNodeInfosEntry + 106, // 102: milvus.proto.query.PartitionLoadInfo.field_indexID:type_name -> milvus.proto.query.PartitionLoadInfo.FieldIndexIDEntry + 107, // 103: milvus.proto.query.Replica.channel_node_infos:type_name -> milvus.proto.query.Replica.ChannelNodeInfosEntry 6, // 104: milvus.proto.query.SyncAction.type:type_name -> milvus.proto.query.SyncType 28, // 105: milvus.proto.query.SyncAction.info:type_name -> milvus.proto.query.SegmentLoadInfo - 121, // 106: milvus.proto.query.SyncAction.checkpoint:type_name -> milvus.proto.msg.MsgPosition - 107, // 107: milvus.proto.query.SyncAction.partition_stats_versions:type_name -> milvus.proto.query.SyncAction.PartitionStatsVersionsEntry - 121, // 108: milvus.proto.query.SyncAction.deleteCP:type_name -> milvus.proto.msg.MsgPosition - 111, // 109: milvus.proto.query.SyncDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 122, // 106: milvus.proto.query.SyncAction.checkpoint:type_name -> milvus.proto.msg.MsgPosition + 108, // 107: milvus.proto.query.SyncAction.partition_stats_versions:type_name -> milvus.proto.query.SyncAction.PartitionStatsVersionsEntry + 122, // 108: milvus.proto.query.SyncAction.deleteCP:type_name -> milvus.proto.msg.MsgPosition + 112, // 109: milvus.proto.query.SyncDistributionRequest.base:type_name -> milvus.proto.common.MsgBase 59, // 110: milvus.proto.query.SyncDistributionRequest.actions:type_name -> milvus.proto.query.SyncAction - 114, // 111: milvus.proto.query.SyncDistributionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 115, // 111: milvus.proto.query.SyncDistributionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema 25, // 112: milvus.proto.query.SyncDistributionRequest.load_meta:type_name -> milvus.proto.query.LoadMetaInfo - 116, // 113: milvus.proto.query.SyncDistributionRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo - 126, // 114: milvus.proto.query.ResourceGroup.config:type_name -> milvus.proto.rg.ResourceGroupConfig - 111, // 115: milvus.proto.query.TransferReplicaRequest.base:type_name -> milvus.proto.common.MsgBase - 111, // 116: milvus.proto.query.DescribeResourceGroupRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 117: milvus.proto.query.DescribeResourceGroupResponse.status:type_name -> milvus.proto.common.Status + 117, // 113: milvus.proto.query.SyncDistributionRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 127, // 114: milvus.proto.query.ResourceGroup.config:type_name -> milvus.proto.rg.ResourceGroupConfig + 112, // 115: milvus.proto.query.TransferReplicaRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 116: milvus.proto.query.DescribeResourceGroupRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 117: milvus.proto.query.DescribeResourceGroupResponse.status:type_name -> milvus.proto.common.Status 65, // 118: milvus.proto.query.DescribeResourceGroupResponse.resource_group:type_name -> milvus.proto.query.ResourceGroupInfo - 108, // 119: milvus.proto.query.ResourceGroupInfo.num_loaded_replica:type_name -> milvus.proto.query.ResourceGroupInfo.NumLoadedReplicaEntry - 109, // 120: milvus.proto.query.ResourceGroupInfo.num_outgoing_node:type_name -> milvus.proto.query.ResourceGroupInfo.NumOutgoingNodeEntry - 110, // 121: milvus.proto.query.ResourceGroupInfo.num_incoming_node:type_name -> milvus.proto.query.ResourceGroupInfo.NumIncomingNodeEntry - 126, // 122: milvus.proto.query.ResourceGroupInfo.config:type_name -> milvus.proto.rg.ResourceGroupConfig - 127, // 123: milvus.proto.query.ResourceGroupInfo.nodes:type_name -> milvus.proto.common.NodeInfo - 111, // 124: milvus.proto.query.DeleteRequest.base:type_name -> milvus.proto.common.MsgBase - 128, // 125: milvus.proto.query.DeleteRequest.primary_keys:type_name -> milvus.proto.schema.IDs + 109, // 119: milvus.proto.query.ResourceGroupInfo.num_loaded_replica:type_name -> milvus.proto.query.ResourceGroupInfo.NumLoadedReplicaEntry + 110, // 120: milvus.proto.query.ResourceGroupInfo.num_outgoing_node:type_name -> milvus.proto.query.ResourceGroupInfo.NumOutgoingNodeEntry + 111, // 121: milvus.proto.query.ResourceGroupInfo.num_incoming_node:type_name -> milvus.proto.query.ResourceGroupInfo.NumIncomingNodeEntry + 127, // 122: milvus.proto.query.ResourceGroupInfo.config:type_name -> milvus.proto.rg.ResourceGroupConfig + 128, // 123: milvus.proto.query.ResourceGroupInfo.nodes:type_name -> milvus.proto.common.NodeInfo + 112, // 124: milvus.proto.query.DeleteRequest.base:type_name -> milvus.proto.common.MsgBase + 129, // 125: milvus.proto.query.DeleteRequest.primary_keys:type_name -> milvus.proto.schema.IDs 1, // 126: milvus.proto.query.DeleteRequest.scope:type_name -> milvus.proto.query.DataScope - 111, // 127: milvus.proto.query.DeleteBatchRequest.base:type_name -> milvus.proto.common.MsgBase - 128, // 128: milvus.proto.query.DeleteBatchRequest.primary_keys:type_name -> milvus.proto.schema.IDs + 112, // 127: milvus.proto.query.DeleteBatchRequest.base:type_name -> milvus.proto.common.MsgBase + 129, // 128: milvus.proto.query.DeleteBatchRequest.primary_keys:type_name -> milvus.proto.schema.IDs 1, // 129: milvus.proto.query.DeleteBatchRequest.scope:type_name -> milvus.proto.query.DataScope - 112, // 130: milvus.proto.query.DeleteBatchResponse.status:type_name -> milvus.proto.common.Status - 111, // 131: milvus.proto.query.ActivateCheckerRequest.base:type_name -> milvus.proto.common.MsgBase - 111, // 132: milvus.proto.query.DeactivateCheckerRequest.base:type_name -> milvus.proto.common.MsgBase - 111, // 133: milvus.proto.query.ListCheckersRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 134: milvus.proto.query.ListCheckersResponse.status:type_name -> milvus.proto.common.Status + 113, // 130: milvus.proto.query.DeleteBatchResponse.status:type_name -> milvus.proto.common.Status + 112, // 131: milvus.proto.query.ActivateCheckerRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 132: milvus.proto.query.DeactivateCheckerRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 133: milvus.proto.query.ListCheckersRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 134: milvus.proto.query.ListCheckersResponse.status:type_name -> milvus.proto.common.Status 73, // 135: milvus.proto.query.ListCheckersResponse.checkerInfos:type_name -> milvus.proto.query.CheckerInfo - 122, // 136: milvus.proto.query.SegmentTarget.level:type_name -> milvus.proto.data.SegmentLevel + 123, // 136: milvus.proto.query.SegmentTarget.level:type_name -> milvus.proto.data.SegmentLevel 74, // 137: milvus.proto.query.PartitionTarget.segments:type_name -> milvus.proto.query.SegmentTarget 75, // 138: milvus.proto.query.ChannelTarget.partition_targets:type_name -> milvus.proto.query.PartitionTarget - 121, // 139: milvus.proto.query.ChannelTarget.seek_position:type_name -> milvus.proto.msg.MsgPosition - 121, // 140: milvus.proto.query.ChannelTarget.delete_checkpoint:type_name -> milvus.proto.msg.MsgPosition + 122, // 139: milvus.proto.query.ChannelTarget.seek_position:type_name -> milvus.proto.msg.MsgPosition + 122, // 140: milvus.proto.query.ChannelTarget.delete_checkpoint:type_name -> milvus.proto.msg.MsgPosition 76, // 141: milvus.proto.query.CollectionTarget.Channel_targets:type_name -> milvus.proto.query.ChannelTarget - 111, // 142: milvus.proto.query.ListQueryNodeRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 143: milvus.proto.query.ListQueryNodeResponse.status:type_name -> milvus.proto.common.Status + 112, // 142: milvus.proto.query.ListQueryNodeRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 143: milvus.proto.query.ListQueryNodeResponse.status:type_name -> milvus.proto.common.Status 78, // 144: milvus.proto.query.ListQueryNodeResponse.nodeInfos:type_name -> milvus.proto.query.NodeInfo - 111, // 145: milvus.proto.query.GetQueryNodeDistributionRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 146: milvus.proto.query.GetQueryNodeDistributionResponse.status:type_name -> milvus.proto.common.Status - 111, // 147: milvus.proto.query.SuspendBalanceRequest.base:type_name -> milvus.proto.common.MsgBase - 111, // 148: milvus.proto.query.ResumeBalanceRequest.base:type_name -> milvus.proto.common.MsgBase - 111, // 149: milvus.proto.query.CheckBalanceStatusRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 150: milvus.proto.query.CheckBalanceStatusResponse.status:type_name -> milvus.proto.common.Status - 111, // 151: milvus.proto.query.SuspendNodeRequest.base:type_name -> milvus.proto.common.MsgBase - 111, // 152: milvus.proto.query.ResumeNodeRequest.base:type_name -> milvus.proto.common.MsgBase - 111, // 153: milvus.proto.query.TransferSegmentRequest.base:type_name -> milvus.proto.common.MsgBase - 111, // 154: milvus.proto.query.TransferChannelRequest.base:type_name -> milvus.proto.common.MsgBase - 111, // 155: milvus.proto.query.CheckQueryNodeDistributionRequest.base:type_name -> milvus.proto.common.MsgBase - 111, // 156: milvus.proto.query.UpdateLoadConfigRequest.base:type_name -> milvus.proto.common.MsgBase - 126, // 157: milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry.value:type_name -> milvus.proto.rg.ResourceGroupConfig - 119, // 158: milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry.value:type_name -> milvus.proto.data.SegmentInfo - 129, // 159: milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats - 130, // 160: milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry.value:type_name -> milvus.proto.data.JsonKeyStats - 121, // 161: milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry.value:type_name -> milvus.proto.msg.MsgPosition - 52, // 162: milvus.proto.query.LeaderView.SegmentDistEntry.value:type_name -> milvus.proto.query.SegmentDist - 121, // 163: milvus.proto.query.LeaderView.GrowingSegmentsEntry.value:type_name -> milvus.proto.msg.MsgPosition - 29, // 164: milvus.proto.query.SegmentVersionInfo.IndexInfoEntry.value:type_name -> milvus.proto.query.FieldIndexInfo - 57, // 165: milvus.proto.query.Replica.ChannelNodeInfosEntry.value:type_name -> milvus.proto.query.ChannelNodeInfo - 7, // 166: milvus.proto.query.QueryCoord.ShowLoadCollections:input_type -> milvus.proto.query.ShowCollectionsRequest - 9, // 167: milvus.proto.query.QueryCoord.ShowLoadPartitions:input_type -> milvus.proto.query.ShowPartitionsRequest - 14, // 168: milvus.proto.query.QueryCoord.LoadPartitions:input_type -> milvus.proto.query.LoadPartitionsRequest - 15, // 169: milvus.proto.query.QueryCoord.ReleasePartitions:input_type -> milvus.proto.query.ReleasePartitionsRequest - 11, // 170: milvus.proto.query.QueryCoord.LoadCollection:input_type -> milvus.proto.query.LoadCollectionRequest - 12, // 171: milvus.proto.query.QueryCoord.ReleaseCollection:input_type -> milvus.proto.query.ReleaseCollectionRequest - 24, // 172: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:input_type -> milvus.proto.query.SyncNewCreatedPartitionRequest - 16, // 173: milvus.proto.query.QueryCoord.GetPartitionStates:input_type -> milvus.proto.query.GetPartitionStatesRequest - 18, // 174: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:input_type -> milvus.proto.query.GetSegmentInfoRequest - 39, // 175: milvus.proto.query.QueryCoord.LoadBalance:input_type -> milvus.proto.query.LoadBalanceRequest - 131, // 176: milvus.proto.query.QueryCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest - 132, // 177: milvus.proto.query.QueryCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest - 133, // 178: milvus.proto.query.QueryCoord.GetReplicas:input_type -> milvus.proto.milvus.GetReplicasRequest - 20, // 179: milvus.proto.query.QueryCoord.GetShardLeaders:input_type -> milvus.proto.query.GetShardLeadersRequest - 134, // 180: milvus.proto.query.QueryCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest - 135, // 181: milvus.proto.query.QueryCoord.CreateResourceGroup:input_type -> milvus.proto.milvus.CreateResourceGroupRequest - 22, // 182: milvus.proto.query.QueryCoord.UpdateResourceGroups:input_type -> milvus.proto.query.UpdateResourceGroupsRequest - 136, // 183: milvus.proto.query.QueryCoord.DropResourceGroup:input_type -> milvus.proto.milvus.DropResourceGroupRequest - 137, // 184: milvus.proto.query.QueryCoord.TransferNode:input_type -> milvus.proto.milvus.TransferNodeRequest - 62, // 185: milvus.proto.query.QueryCoord.TransferReplica:input_type -> milvus.proto.query.TransferReplicaRequest - 138, // 186: milvus.proto.query.QueryCoord.ListResourceGroups:input_type -> milvus.proto.milvus.ListResourceGroupsRequest - 63, // 187: milvus.proto.query.QueryCoord.DescribeResourceGroup:input_type -> milvus.proto.query.DescribeResourceGroupRequest - 71, // 188: milvus.proto.query.QueryCoord.ListCheckers:input_type -> milvus.proto.query.ListCheckersRequest - 69, // 189: milvus.proto.query.QueryCoord.ActivateChecker:input_type -> milvus.proto.query.ActivateCheckerRequest - 70, // 190: milvus.proto.query.QueryCoord.DeactivateChecker:input_type -> milvus.proto.query.DeactivateCheckerRequest - 79, // 191: milvus.proto.query.QueryCoord.ListQueryNode:input_type -> milvus.proto.query.ListQueryNodeRequest - 81, // 192: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:input_type -> milvus.proto.query.GetQueryNodeDistributionRequest - 83, // 193: milvus.proto.query.QueryCoord.SuspendBalance:input_type -> milvus.proto.query.SuspendBalanceRequest - 84, // 194: milvus.proto.query.QueryCoord.ResumeBalance:input_type -> milvus.proto.query.ResumeBalanceRequest - 85, // 195: milvus.proto.query.QueryCoord.CheckBalanceStatus:input_type -> milvus.proto.query.CheckBalanceStatusRequest - 87, // 196: milvus.proto.query.QueryCoord.SuspendNode:input_type -> milvus.proto.query.SuspendNodeRequest - 88, // 197: milvus.proto.query.QueryCoord.ResumeNode:input_type -> milvus.proto.query.ResumeNodeRequest - 89, // 198: milvus.proto.query.QueryCoord.TransferSegment:input_type -> milvus.proto.query.TransferSegmentRequest - 90, // 199: milvus.proto.query.QueryCoord.TransferChannel:input_type -> milvus.proto.query.TransferChannelRequest - 91, // 200: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:input_type -> milvus.proto.query.CheckQueryNodeDistributionRequest - 92, // 201: milvus.proto.query.QueryCoord.UpdateLoadConfig:input_type -> milvus.proto.query.UpdateLoadConfigRequest - 139, // 202: milvus.proto.query.QueryNode.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest - 140, // 203: milvus.proto.query.QueryNode.GetTimeTickChannel:input_type -> milvus.proto.internal.GetTimeTickChannelRequest - 141, // 204: milvus.proto.query.QueryNode.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest - 26, // 205: milvus.proto.query.QueryNode.WatchDmChannels:input_type -> milvus.proto.query.WatchDmChannelsRequest - 27, // 206: milvus.proto.query.QueryNode.UnsubDmChannel:input_type -> milvus.proto.query.UnsubDmChannelRequest - 30, // 207: milvus.proto.query.QueryNode.LoadSegments:input_type -> milvus.proto.query.LoadSegmentsRequest - 12, // 208: milvus.proto.query.QueryNode.ReleaseCollection:input_type -> milvus.proto.query.ReleaseCollectionRequest - 14, // 209: milvus.proto.query.QueryNode.LoadPartitions:input_type -> milvus.proto.query.LoadPartitionsRequest - 15, // 210: milvus.proto.query.QueryNode.ReleasePartitions:input_type -> milvus.proto.query.ReleasePartitionsRequest - 31, // 211: milvus.proto.query.QueryNode.ReleaseSegments:input_type -> milvus.proto.query.ReleaseSegmentsRequest - 18, // 212: milvus.proto.query.QueryNode.GetSegmentInfo:input_type -> milvus.proto.query.GetSegmentInfoRequest - 34, // 213: milvus.proto.query.QueryNode.SyncReplicaSegments:input_type -> milvus.proto.query.SyncReplicaSegmentsRequest - 13, // 214: milvus.proto.query.QueryNode.GetStatistics:input_type -> milvus.proto.query.GetStatisticsRequest - 32, // 215: milvus.proto.query.QueryNode.Search:input_type -> milvus.proto.query.SearchRequest - 32, // 216: milvus.proto.query.QueryNode.SearchSegments:input_type -> milvus.proto.query.SearchRequest - 33, // 217: milvus.proto.query.QueryNode.Query:input_type -> milvus.proto.query.QueryRequest - 33, // 218: milvus.proto.query.QueryNode.QueryStream:input_type -> milvus.proto.query.QueryRequest - 33, // 219: milvus.proto.query.QueryNode.QuerySegments:input_type -> milvus.proto.query.QueryRequest - 33, // 220: milvus.proto.query.QueryNode.QueryStreamSegments:input_type -> milvus.proto.query.QueryRequest - 131, // 221: milvus.proto.query.QueryNode.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest - 132, // 222: milvus.proto.query.QueryNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest - 49, // 223: milvus.proto.query.QueryNode.GetDataDistribution:input_type -> milvus.proto.query.GetDataDistributionRequest - 60, // 224: milvus.proto.query.QueryNode.SyncDistribution:input_type -> milvus.proto.query.SyncDistributionRequest - 66, // 225: milvus.proto.query.QueryNode.Delete:input_type -> milvus.proto.query.DeleteRequest - 67, // 226: milvus.proto.query.QueryNode.DeleteBatch:input_type -> milvus.proto.query.DeleteBatchRequest - 8, // 227: milvus.proto.query.QueryCoord.ShowLoadCollections:output_type -> milvus.proto.query.ShowCollectionsResponse - 10, // 228: milvus.proto.query.QueryCoord.ShowLoadPartitions:output_type -> milvus.proto.query.ShowPartitionsResponse - 112, // 229: milvus.proto.query.QueryCoord.LoadPartitions:output_type -> milvus.proto.common.Status - 112, // 230: milvus.proto.query.QueryCoord.ReleasePartitions:output_type -> milvus.proto.common.Status - 112, // 231: milvus.proto.query.QueryCoord.LoadCollection:output_type -> milvus.proto.common.Status - 112, // 232: milvus.proto.query.QueryCoord.ReleaseCollection:output_type -> milvus.proto.common.Status - 112, // 233: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:output_type -> milvus.proto.common.Status - 17, // 234: milvus.proto.query.QueryCoord.GetPartitionStates:output_type -> milvus.proto.query.GetPartitionStatesResponse - 19, // 235: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse - 112, // 236: milvus.proto.query.QueryCoord.LoadBalance:output_type -> milvus.proto.common.Status - 142, // 237: milvus.proto.query.QueryCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 143, // 238: milvus.proto.query.QueryCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 144, // 239: milvus.proto.query.QueryCoord.GetReplicas:output_type -> milvus.proto.milvus.GetReplicasResponse - 21, // 240: milvus.proto.query.QueryCoord.GetShardLeaders:output_type -> milvus.proto.query.GetShardLeadersResponse - 145, // 241: milvus.proto.query.QueryCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse - 112, // 242: milvus.proto.query.QueryCoord.CreateResourceGroup:output_type -> milvus.proto.common.Status - 112, // 243: milvus.proto.query.QueryCoord.UpdateResourceGroups:output_type -> milvus.proto.common.Status - 112, // 244: milvus.proto.query.QueryCoord.DropResourceGroup:output_type -> milvus.proto.common.Status - 112, // 245: milvus.proto.query.QueryCoord.TransferNode:output_type -> milvus.proto.common.Status - 112, // 246: milvus.proto.query.QueryCoord.TransferReplica:output_type -> milvus.proto.common.Status - 146, // 247: milvus.proto.query.QueryCoord.ListResourceGroups:output_type -> milvus.proto.milvus.ListResourceGroupsResponse - 64, // 248: milvus.proto.query.QueryCoord.DescribeResourceGroup:output_type -> milvus.proto.query.DescribeResourceGroupResponse - 72, // 249: milvus.proto.query.QueryCoord.ListCheckers:output_type -> milvus.proto.query.ListCheckersResponse - 112, // 250: milvus.proto.query.QueryCoord.ActivateChecker:output_type -> milvus.proto.common.Status - 112, // 251: milvus.proto.query.QueryCoord.DeactivateChecker:output_type -> milvus.proto.common.Status - 80, // 252: milvus.proto.query.QueryCoord.ListQueryNode:output_type -> milvus.proto.query.ListQueryNodeResponse - 82, // 253: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:output_type -> milvus.proto.query.GetQueryNodeDistributionResponse - 112, // 254: milvus.proto.query.QueryCoord.SuspendBalance:output_type -> milvus.proto.common.Status - 112, // 255: milvus.proto.query.QueryCoord.ResumeBalance:output_type -> milvus.proto.common.Status - 86, // 256: milvus.proto.query.QueryCoord.CheckBalanceStatus:output_type -> milvus.proto.query.CheckBalanceStatusResponse - 112, // 257: milvus.proto.query.QueryCoord.SuspendNode:output_type -> milvus.proto.common.Status - 112, // 258: milvus.proto.query.QueryCoord.ResumeNode:output_type -> milvus.proto.common.Status - 112, // 259: milvus.proto.query.QueryCoord.TransferSegment:output_type -> milvus.proto.common.Status - 112, // 260: milvus.proto.query.QueryCoord.TransferChannel:output_type -> milvus.proto.common.Status - 112, // 261: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:output_type -> milvus.proto.common.Status - 112, // 262: milvus.proto.query.QueryCoord.UpdateLoadConfig:output_type -> milvus.proto.common.Status - 147, // 263: milvus.proto.query.QueryNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates - 148, // 264: milvus.proto.query.QueryNode.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse - 148, // 265: milvus.proto.query.QueryNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse - 112, // 266: milvus.proto.query.QueryNode.WatchDmChannels:output_type -> milvus.proto.common.Status - 112, // 267: milvus.proto.query.QueryNode.UnsubDmChannel:output_type -> milvus.proto.common.Status - 112, // 268: milvus.proto.query.QueryNode.LoadSegments:output_type -> milvus.proto.common.Status - 112, // 269: milvus.proto.query.QueryNode.ReleaseCollection:output_type -> milvus.proto.common.Status - 112, // 270: milvus.proto.query.QueryNode.LoadPartitions:output_type -> milvus.proto.common.Status - 112, // 271: milvus.proto.query.QueryNode.ReleasePartitions:output_type -> milvus.proto.common.Status - 112, // 272: milvus.proto.query.QueryNode.ReleaseSegments:output_type -> milvus.proto.common.Status - 19, // 273: milvus.proto.query.QueryNode.GetSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse - 112, // 274: milvus.proto.query.QueryNode.SyncReplicaSegments:output_type -> milvus.proto.common.Status - 149, // 275: milvus.proto.query.QueryNode.GetStatistics:output_type -> milvus.proto.internal.GetStatisticsResponse - 150, // 276: milvus.proto.query.QueryNode.Search:output_type -> milvus.proto.internal.SearchResults - 150, // 277: milvus.proto.query.QueryNode.SearchSegments:output_type -> milvus.proto.internal.SearchResults - 151, // 278: milvus.proto.query.QueryNode.Query:output_type -> milvus.proto.internal.RetrieveResults - 151, // 279: milvus.proto.query.QueryNode.QueryStream:output_type -> milvus.proto.internal.RetrieveResults - 151, // 280: milvus.proto.query.QueryNode.QuerySegments:output_type -> milvus.proto.internal.RetrieveResults - 151, // 281: milvus.proto.query.QueryNode.QueryStreamSegments:output_type -> milvus.proto.internal.RetrieveResults - 142, // 282: milvus.proto.query.QueryNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 143, // 283: milvus.proto.query.QueryNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 50, // 284: milvus.proto.query.QueryNode.GetDataDistribution:output_type -> milvus.proto.query.GetDataDistributionResponse - 112, // 285: milvus.proto.query.QueryNode.SyncDistribution:output_type -> milvus.proto.common.Status - 112, // 286: milvus.proto.query.QueryNode.Delete:output_type -> milvus.proto.common.Status - 68, // 287: milvus.proto.query.QueryNode.DeleteBatch:output_type -> milvus.proto.query.DeleteBatchResponse - 227, // [227:288] is the sub-list for method output_type - 166, // [166:227] is the sub-list for method input_type - 166, // [166:166] is the sub-list for extension type_name - 166, // [166:166] is the sub-list for extension extendee - 0, // [0:166] is the sub-list for field type_name + 112, // 145: milvus.proto.query.GetQueryNodeDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 146: milvus.proto.query.GetQueryNodeDistributionResponse.status:type_name -> milvus.proto.common.Status + 112, // 147: milvus.proto.query.SuspendBalanceRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 148: milvus.proto.query.ResumeBalanceRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 149: milvus.proto.query.CheckBalanceStatusRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 150: milvus.proto.query.CheckBalanceStatusResponse.status:type_name -> milvus.proto.common.Status + 112, // 151: milvus.proto.query.SuspendNodeRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 152: milvus.proto.query.ResumeNodeRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 153: milvus.proto.query.TransferSegmentRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 154: milvus.proto.query.TransferChannelRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 155: milvus.proto.query.CheckQueryNodeDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 156: milvus.proto.query.UpdateLoadConfigRequest.base:type_name -> milvus.proto.common.MsgBase + 112, // 157: milvus.proto.query.UpdateSchemaRequest.base:type_name -> milvus.proto.common.MsgBase + 115, // 158: milvus.proto.query.UpdateSchemaRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 127, // 159: milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry.value:type_name -> milvus.proto.rg.ResourceGroupConfig + 120, // 160: milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry.value:type_name -> milvus.proto.data.SegmentInfo + 130, // 161: milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats + 131, // 162: milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry.value:type_name -> milvus.proto.data.JsonKeyStats + 122, // 163: milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry.value:type_name -> milvus.proto.msg.MsgPosition + 52, // 164: milvus.proto.query.LeaderView.SegmentDistEntry.value:type_name -> milvus.proto.query.SegmentDist + 122, // 165: milvus.proto.query.LeaderView.GrowingSegmentsEntry.value:type_name -> milvus.proto.msg.MsgPosition + 29, // 166: milvus.proto.query.SegmentVersionInfo.IndexInfoEntry.value:type_name -> milvus.proto.query.FieldIndexInfo + 57, // 167: milvus.proto.query.Replica.ChannelNodeInfosEntry.value:type_name -> milvus.proto.query.ChannelNodeInfo + 7, // 168: milvus.proto.query.QueryCoord.ShowLoadCollections:input_type -> milvus.proto.query.ShowCollectionsRequest + 9, // 169: milvus.proto.query.QueryCoord.ShowLoadPartitions:input_type -> milvus.proto.query.ShowPartitionsRequest + 14, // 170: milvus.proto.query.QueryCoord.LoadPartitions:input_type -> milvus.proto.query.LoadPartitionsRequest + 15, // 171: milvus.proto.query.QueryCoord.ReleasePartitions:input_type -> milvus.proto.query.ReleasePartitionsRequest + 11, // 172: milvus.proto.query.QueryCoord.LoadCollection:input_type -> milvus.proto.query.LoadCollectionRequest + 12, // 173: milvus.proto.query.QueryCoord.ReleaseCollection:input_type -> milvus.proto.query.ReleaseCollectionRequest + 24, // 174: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:input_type -> milvus.proto.query.SyncNewCreatedPartitionRequest + 16, // 175: milvus.proto.query.QueryCoord.GetPartitionStates:input_type -> milvus.proto.query.GetPartitionStatesRequest + 18, // 176: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:input_type -> milvus.proto.query.GetSegmentInfoRequest + 39, // 177: milvus.proto.query.QueryCoord.LoadBalance:input_type -> milvus.proto.query.LoadBalanceRequest + 132, // 178: milvus.proto.query.QueryCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 133, // 179: milvus.proto.query.QueryCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 134, // 180: milvus.proto.query.QueryCoord.GetReplicas:input_type -> milvus.proto.milvus.GetReplicasRequest + 20, // 181: milvus.proto.query.QueryCoord.GetShardLeaders:input_type -> milvus.proto.query.GetShardLeadersRequest + 135, // 182: milvus.proto.query.QueryCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest + 136, // 183: milvus.proto.query.QueryCoord.CreateResourceGroup:input_type -> milvus.proto.milvus.CreateResourceGroupRequest + 22, // 184: milvus.proto.query.QueryCoord.UpdateResourceGroups:input_type -> milvus.proto.query.UpdateResourceGroupsRequest + 137, // 185: milvus.proto.query.QueryCoord.DropResourceGroup:input_type -> milvus.proto.milvus.DropResourceGroupRequest + 138, // 186: milvus.proto.query.QueryCoord.TransferNode:input_type -> milvus.proto.milvus.TransferNodeRequest + 62, // 187: milvus.proto.query.QueryCoord.TransferReplica:input_type -> milvus.proto.query.TransferReplicaRequest + 139, // 188: milvus.proto.query.QueryCoord.ListResourceGroups:input_type -> milvus.proto.milvus.ListResourceGroupsRequest + 63, // 189: milvus.proto.query.QueryCoord.DescribeResourceGroup:input_type -> milvus.proto.query.DescribeResourceGroupRequest + 71, // 190: milvus.proto.query.QueryCoord.ListCheckers:input_type -> milvus.proto.query.ListCheckersRequest + 69, // 191: milvus.proto.query.QueryCoord.ActivateChecker:input_type -> milvus.proto.query.ActivateCheckerRequest + 70, // 192: milvus.proto.query.QueryCoord.DeactivateChecker:input_type -> milvus.proto.query.DeactivateCheckerRequest + 79, // 193: milvus.proto.query.QueryCoord.ListQueryNode:input_type -> milvus.proto.query.ListQueryNodeRequest + 81, // 194: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:input_type -> milvus.proto.query.GetQueryNodeDistributionRequest + 83, // 195: milvus.proto.query.QueryCoord.SuspendBalance:input_type -> milvus.proto.query.SuspendBalanceRequest + 84, // 196: milvus.proto.query.QueryCoord.ResumeBalance:input_type -> milvus.proto.query.ResumeBalanceRequest + 85, // 197: milvus.proto.query.QueryCoord.CheckBalanceStatus:input_type -> milvus.proto.query.CheckBalanceStatusRequest + 87, // 198: milvus.proto.query.QueryCoord.SuspendNode:input_type -> milvus.proto.query.SuspendNodeRequest + 88, // 199: milvus.proto.query.QueryCoord.ResumeNode:input_type -> milvus.proto.query.ResumeNodeRequest + 89, // 200: milvus.proto.query.QueryCoord.TransferSegment:input_type -> milvus.proto.query.TransferSegmentRequest + 90, // 201: milvus.proto.query.QueryCoord.TransferChannel:input_type -> milvus.proto.query.TransferChannelRequest + 91, // 202: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:input_type -> milvus.proto.query.CheckQueryNodeDistributionRequest + 92, // 203: milvus.proto.query.QueryCoord.UpdateLoadConfig:input_type -> milvus.proto.query.UpdateLoadConfigRequest + 140, // 204: milvus.proto.query.QueryNode.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 141, // 205: milvus.proto.query.QueryNode.GetTimeTickChannel:input_type -> milvus.proto.internal.GetTimeTickChannelRequest + 142, // 206: milvus.proto.query.QueryNode.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 26, // 207: milvus.proto.query.QueryNode.WatchDmChannels:input_type -> milvus.proto.query.WatchDmChannelsRequest + 27, // 208: milvus.proto.query.QueryNode.UnsubDmChannel:input_type -> milvus.proto.query.UnsubDmChannelRequest + 30, // 209: milvus.proto.query.QueryNode.LoadSegments:input_type -> milvus.proto.query.LoadSegmentsRequest + 12, // 210: milvus.proto.query.QueryNode.ReleaseCollection:input_type -> milvus.proto.query.ReleaseCollectionRequest + 14, // 211: milvus.proto.query.QueryNode.LoadPartitions:input_type -> milvus.proto.query.LoadPartitionsRequest + 15, // 212: milvus.proto.query.QueryNode.ReleasePartitions:input_type -> milvus.proto.query.ReleasePartitionsRequest + 31, // 213: milvus.proto.query.QueryNode.ReleaseSegments:input_type -> milvus.proto.query.ReleaseSegmentsRequest + 18, // 214: milvus.proto.query.QueryNode.GetSegmentInfo:input_type -> milvus.proto.query.GetSegmentInfoRequest + 34, // 215: milvus.proto.query.QueryNode.SyncReplicaSegments:input_type -> milvus.proto.query.SyncReplicaSegmentsRequest + 13, // 216: milvus.proto.query.QueryNode.GetStatistics:input_type -> milvus.proto.query.GetStatisticsRequest + 32, // 217: milvus.proto.query.QueryNode.Search:input_type -> milvus.proto.query.SearchRequest + 32, // 218: milvus.proto.query.QueryNode.SearchSegments:input_type -> milvus.proto.query.SearchRequest + 33, // 219: milvus.proto.query.QueryNode.Query:input_type -> milvus.proto.query.QueryRequest + 33, // 220: milvus.proto.query.QueryNode.QueryStream:input_type -> milvus.proto.query.QueryRequest + 33, // 221: milvus.proto.query.QueryNode.QuerySegments:input_type -> milvus.proto.query.QueryRequest + 33, // 222: milvus.proto.query.QueryNode.QueryStreamSegments:input_type -> milvus.proto.query.QueryRequest + 132, // 223: milvus.proto.query.QueryNode.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 133, // 224: milvus.proto.query.QueryNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 49, // 225: milvus.proto.query.QueryNode.GetDataDistribution:input_type -> milvus.proto.query.GetDataDistributionRequest + 60, // 226: milvus.proto.query.QueryNode.SyncDistribution:input_type -> milvus.proto.query.SyncDistributionRequest + 66, // 227: milvus.proto.query.QueryNode.Delete:input_type -> milvus.proto.query.DeleteRequest + 67, // 228: milvus.proto.query.QueryNode.DeleteBatch:input_type -> milvus.proto.query.DeleteBatchRequest + 93, // 229: milvus.proto.query.QueryNode.UpdateSchema:input_type -> milvus.proto.query.UpdateSchemaRequest + 8, // 230: milvus.proto.query.QueryCoord.ShowLoadCollections:output_type -> milvus.proto.query.ShowCollectionsResponse + 10, // 231: milvus.proto.query.QueryCoord.ShowLoadPartitions:output_type -> milvus.proto.query.ShowPartitionsResponse + 113, // 232: milvus.proto.query.QueryCoord.LoadPartitions:output_type -> milvus.proto.common.Status + 113, // 233: milvus.proto.query.QueryCoord.ReleasePartitions:output_type -> milvus.proto.common.Status + 113, // 234: milvus.proto.query.QueryCoord.LoadCollection:output_type -> milvus.proto.common.Status + 113, // 235: milvus.proto.query.QueryCoord.ReleaseCollection:output_type -> milvus.proto.common.Status + 113, // 236: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:output_type -> milvus.proto.common.Status + 17, // 237: milvus.proto.query.QueryCoord.GetPartitionStates:output_type -> milvus.proto.query.GetPartitionStatesResponse + 19, // 238: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse + 113, // 239: milvus.proto.query.QueryCoord.LoadBalance:output_type -> milvus.proto.common.Status + 143, // 240: milvus.proto.query.QueryCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 144, // 241: milvus.proto.query.QueryCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 145, // 242: milvus.proto.query.QueryCoord.GetReplicas:output_type -> milvus.proto.milvus.GetReplicasResponse + 21, // 243: milvus.proto.query.QueryCoord.GetShardLeaders:output_type -> milvus.proto.query.GetShardLeadersResponse + 146, // 244: milvus.proto.query.QueryCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse + 113, // 245: milvus.proto.query.QueryCoord.CreateResourceGroup:output_type -> milvus.proto.common.Status + 113, // 246: milvus.proto.query.QueryCoord.UpdateResourceGroups:output_type -> milvus.proto.common.Status + 113, // 247: milvus.proto.query.QueryCoord.DropResourceGroup:output_type -> milvus.proto.common.Status + 113, // 248: milvus.proto.query.QueryCoord.TransferNode:output_type -> milvus.proto.common.Status + 113, // 249: milvus.proto.query.QueryCoord.TransferReplica:output_type -> milvus.proto.common.Status + 147, // 250: milvus.proto.query.QueryCoord.ListResourceGroups:output_type -> milvus.proto.milvus.ListResourceGroupsResponse + 64, // 251: milvus.proto.query.QueryCoord.DescribeResourceGroup:output_type -> milvus.proto.query.DescribeResourceGroupResponse + 72, // 252: milvus.proto.query.QueryCoord.ListCheckers:output_type -> milvus.proto.query.ListCheckersResponse + 113, // 253: milvus.proto.query.QueryCoord.ActivateChecker:output_type -> milvus.proto.common.Status + 113, // 254: milvus.proto.query.QueryCoord.DeactivateChecker:output_type -> milvus.proto.common.Status + 80, // 255: milvus.proto.query.QueryCoord.ListQueryNode:output_type -> milvus.proto.query.ListQueryNodeResponse + 82, // 256: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:output_type -> milvus.proto.query.GetQueryNodeDistributionResponse + 113, // 257: milvus.proto.query.QueryCoord.SuspendBalance:output_type -> milvus.proto.common.Status + 113, // 258: milvus.proto.query.QueryCoord.ResumeBalance:output_type -> milvus.proto.common.Status + 86, // 259: milvus.proto.query.QueryCoord.CheckBalanceStatus:output_type -> milvus.proto.query.CheckBalanceStatusResponse + 113, // 260: milvus.proto.query.QueryCoord.SuspendNode:output_type -> milvus.proto.common.Status + 113, // 261: milvus.proto.query.QueryCoord.ResumeNode:output_type -> milvus.proto.common.Status + 113, // 262: milvus.proto.query.QueryCoord.TransferSegment:output_type -> milvus.proto.common.Status + 113, // 263: milvus.proto.query.QueryCoord.TransferChannel:output_type -> milvus.proto.common.Status + 113, // 264: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:output_type -> milvus.proto.common.Status + 113, // 265: milvus.proto.query.QueryCoord.UpdateLoadConfig:output_type -> milvus.proto.common.Status + 148, // 266: milvus.proto.query.QueryNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 149, // 267: milvus.proto.query.QueryNode.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse + 149, // 268: milvus.proto.query.QueryNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 113, // 269: milvus.proto.query.QueryNode.WatchDmChannels:output_type -> milvus.proto.common.Status + 113, // 270: milvus.proto.query.QueryNode.UnsubDmChannel:output_type -> milvus.proto.common.Status + 113, // 271: milvus.proto.query.QueryNode.LoadSegments:output_type -> milvus.proto.common.Status + 113, // 272: milvus.proto.query.QueryNode.ReleaseCollection:output_type -> milvus.proto.common.Status + 113, // 273: milvus.proto.query.QueryNode.LoadPartitions:output_type -> milvus.proto.common.Status + 113, // 274: milvus.proto.query.QueryNode.ReleasePartitions:output_type -> milvus.proto.common.Status + 113, // 275: milvus.proto.query.QueryNode.ReleaseSegments:output_type -> milvus.proto.common.Status + 19, // 276: milvus.proto.query.QueryNode.GetSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse + 113, // 277: milvus.proto.query.QueryNode.SyncReplicaSegments:output_type -> milvus.proto.common.Status + 150, // 278: milvus.proto.query.QueryNode.GetStatistics:output_type -> milvus.proto.internal.GetStatisticsResponse + 151, // 279: milvus.proto.query.QueryNode.Search:output_type -> milvus.proto.internal.SearchResults + 151, // 280: milvus.proto.query.QueryNode.SearchSegments:output_type -> milvus.proto.internal.SearchResults + 152, // 281: milvus.proto.query.QueryNode.Query:output_type -> milvus.proto.internal.RetrieveResults + 152, // 282: milvus.proto.query.QueryNode.QueryStream:output_type -> milvus.proto.internal.RetrieveResults + 152, // 283: milvus.proto.query.QueryNode.QuerySegments:output_type -> milvus.proto.internal.RetrieveResults + 152, // 284: milvus.proto.query.QueryNode.QueryStreamSegments:output_type -> milvus.proto.internal.RetrieveResults + 143, // 285: milvus.proto.query.QueryNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 144, // 286: milvus.proto.query.QueryNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 50, // 287: milvus.proto.query.QueryNode.GetDataDistribution:output_type -> milvus.proto.query.GetDataDistributionResponse + 113, // 288: milvus.proto.query.QueryNode.SyncDistribution:output_type -> milvus.proto.common.Status + 113, // 289: milvus.proto.query.QueryNode.Delete:output_type -> milvus.proto.common.Status + 68, // 290: milvus.proto.query.QueryNode.DeleteBatch:output_type -> milvus.proto.query.DeleteBatchResponse + 113, // 291: milvus.proto.query.QueryNode.UpdateSchema:output_type -> milvus.proto.common.Status + 230, // [230:292] is the sub-list for method output_type + 168, // [168:230] is the sub-list for method input_type + 168, // [168:168] is the sub-list for extension type_name + 168, // [168:168] is the sub-list for extension extendee + 0, // [0:168] is the sub-list for field type_name } func init() { file_query_coord_proto_init() } @@ -10592,6 +10677,18 @@ func file_query_coord_proto_init() { return nil } } + file_query_coord_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateSchemaRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } type x struct{} out := protoimpl.TypeBuilder{ @@ -10599,7 +10696,7 @@ func file_query_coord_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_query_coord_proto_rawDesc, NumEnums: 7, - NumMessages: 104, + NumMessages: 105, 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 3e0b9d9d4e..e84d685090 100644 --- a/pkg/proto/querypb/query_coord_grpc.pb.go +++ b/pkg/proto/querypb/query_coord_grpc.pb.go @@ -1436,6 +1436,7 @@ const ( QueryNode_SyncDistribution_FullMethodName = "/milvus.proto.query.QueryNode/SyncDistribution" QueryNode_Delete_FullMethodName = "/milvus.proto.query.QueryNode/Delete" QueryNode_DeleteBatch_FullMethodName = "/milvus.proto.query.QueryNode/DeleteBatch" + QueryNode_UpdateSchema_FullMethodName = "/milvus.proto.query.QueryNode/UpdateSchema" ) // QueryNodeClient is the client API for QueryNode service. @@ -1470,6 +1471,7 @@ type QueryNodeClient interface { // DeleteBatch is the API to apply same delete data into multiple segments. // it's basically same as `Delete` but cost less memory pressure. DeleteBatch(ctx context.Context, in *DeleteBatchRequest, opts ...grpc.CallOption) (*DeleteBatchResponse, error) + UpdateSchema(ctx context.Context, in *UpdateSchemaRequest, opts ...grpc.CallOption) (*commonpb.Status, error) } type queryNodeClient struct { @@ -1751,6 +1753,15 @@ func (c *queryNodeClient) DeleteBatch(ctx context.Context, in *DeleteBatchReques return out, nil } +func (c *queryNodeClient) UpdateSchema(ctx context.Context, in *UpdateSchemaRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { + out := new(commonpb.Status) + err := c.cc.Invoke(ctx, QueryNode_UpdateSchema_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // QueryNodeServer is the server API for QueryNode service. // All implementations should embed UnimplementedQueryNodeServer // for forward compatibility @@ -1783,6 +1794,7 @@ type QueryNodeServer interface { // DeleteBatch is the API to apply same delete data into multiple segments. // it's basically same as `Delete` but cost less memory pressure. DeleteBatch(context.Context, *DeleteBatchRequest) (*DeleteBatchResponse, error) + UpdateSchema(context.Context, *UpdateSchemaRequest) (*commonpb.Status, error) } // UnimplementedQueryNodeServer should be embedded to have forward compatible implementations. @@ -1864,6 +1876,9 @@ func (UnimplementedQueryNodeServer) Delete(context.Context, *DeleteRequest) (*co func (UnimplementedQueryNodeServer) DeleteBatch(context.Context, *DeleteBatchRequest) (*DeleteBatchResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method DeleteBatch not implemented") } +func (UnimplementedQueryNodeServer) UpdateSchema(context.Context, *UpdateSchemaRequest) (*commonpb.Status, error) { + return nil, status.Errorf(codes.Unimplemented, "method UpdateSchema not implemented") +} // UnsafeQueryNodeServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to QueryNodeServer will @@ -2332,6 +2347,24 @@ func _QueryNode_DeleteBatch_Handler(srv interface{}, ctx context.Context, dec fu return interceptor(ctx, in, info, handler) } +func _QueryNode_UpdateSchema_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(UpdateSchemaRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(QueryNodeServer).UpdateSchema(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: QueryNode_UpdateSchema_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(QueryNodeServer).UpdateSchema(ctx, req.(*UpdateSchemaRequest)) + } + return interceptor(ctx, in, info, handler) +} + // QueryNode_ServiceDesc is the grpc.ServiceDesc for QueryNode service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) @@ -2431,6 +2464,10 @@ var QueryNode_ServiceDesc = grpc.ServiceDesc{ MethodName: "DeleteBatch", Handler: _QueryNode_DeleteBatch_Handler, }, + { + MethodName: "UpdateSchema", + Handler: _QueryNode_UpdateSchema_Handler, + }, }, Streams: []grpc.StreamDesc{ { diff --git a/pkg/streaming/util/message/adaptor/message.go b/pkg/streaming/util/message/adaptor/message.go index ba03df1aa8..2e21d65ec5 100644 --- a/pkg/streaming/util/message/adaptor/message.go +++ b/pkg/streaming/util/message/adaptor/message.go @@ -135,6 +135,8 @@ func fromMessageToTsMsgV2(msg message.ImmutableMessage) (msgstream.TsMsg, error) tsMsg, err = NewManualFlushMessageBody(msg) case message.MessageTypeCreateSegment: tsMsg, err = NewCreateSegmentMessageBody(msg) + case message.MessageTypeSchemaChange: + tsMsg, err = NewSchemaChangeMessageBody(msg) default: panic("unsupported message type") } diff --git a/pkg/streaming/util/message/adaptor/message_type.go b/pkg/streaming/util/message/adaptor/message_type.go index f567365e1e..e319dd175e 100644 --- a/pkg/streaming/util/message/adaptor/message_type.go +++ b/pkg/streaming/util/message/adaptor/message_type.go @@ -17,6 +17,7 @@ var messageTypeToCommonpbMsgType = map[message.MessageType]commonpb.MsgType{ message.MessageTypeCreatePartition: commonpb.MsgType_CreatePartition, message.MessageTypeDropPartition: commonpb.MsgType_DropPartition, message.MessageTypeImport: commonpb.MsgType_Import, + message.MessageTypeSchemaChange: commonpb.MsgType_AddCollectionField, // TODO change to schema change } // MustGetCommonpbMsgTypeFromMessageType returns the commonpb.MsgType from message.MessageType. diff --git a/pkg/streaming/util/message/adaptor/ts_msg_newer.go b/pkg/streaming/util/message/adaptor/ts_msg_newer.go index 04ed27d4de..984bd2589a 100644 --- a/pkg/streaming/util/message/adaptor/ts_msg_newer.go +++ b/pkg/streaming/util/message/adaptor/ts_msg_newer.go @@ -121,3 +121,29 @@ func NewManualFlushMessageBody(msg message.ImmutableMessage) (msgstream.TsMsg, e ManualFlushMessage: flushMsg, }, nil } + +type SchemaChangeMessageBody struct { + *tsMsgImpl + SchemaChangeMessage message.ImmutableSchemaChangeMessageV2 + BroadcastID uint64 +} + +func NewSchemaChangeMessageBody(msg message.ImmutableMessage) (msgstream.TsMsg, error) { + schChgMsg, err := message.AsImmutableCollectionSchemaChangeV2(msg) + if err != nil { + return nil, err + } + return &SchemaChangeMessageBody{ + tsMsgImpl: &tsMsgImpl{ + BaseMsg: msgstream.BaseMsg{ + BeginTimestamp: msg.TimeTick(), + EndTimestamp: msg.TimeTick(), + }, + ts: msg.TimeTick(), + sz: msg.EstimateSize(), + msgType: MustGetCommonpbMsgTypeFromMessageType(msg.MessageType()), + }, + SchemaChangeMessage: schChgMsg, + BroadcastID: msg.BroadcastHeader().BroadcastID, + }, nil +} diff --git a/pkg/streaming/util/message/broadcast.go b/pkg/streaming/util/message/broadcast.go index 0eabcf839e..84df2eda69 100644 --- a/pkg/streaming/util/message/broadcast.go +++ b/pkg/streaming/util/message/broadcast.go @@ -9,13 +9,13 @@ import ( // newBroadcastHeaderFromProto creates a BroadcastHeader from proto. func newBroadcastHeaderFromProto(proto *messagespb.BroadcastHeader) *BroadcastHeader { - rks := make(typeutil.Set[ResourceKey], len(proto.ResourceKeys)) - for _, key := range proto.ResourceKeys { + rks := make(typeutil.Set[ResourceKey], len(proto.GetResourceKeys())) + for _, key := range proto.GetResourceKeys() { rks.Insert(NewResourceKeyFromProto(key)) } return &BroadcastHeader{ - BroadcastID: proto.BroadcastId, - VChannels: proto.Vchannels, + BroadcastID: proto.GetBroadcastId(), + VChannels: proto.GetVchannels(), ResourceKeys: rks, } } diff --git a/pkg/streaming/util/message/builder.go b/pkg/streaming/util/message/builder.go index ddc85edb87..736575993f 100644 --- a/pkg/streaming/util/message/builder.go +++ b/pkg/streaming/util/message/builder.go @@ -68,6 +68,7 @@ var ( NewBeginTxnMessageBuilderV2 = createNewMessageBuilderV2[*BeginTxnMessageHeader, *BeginTxnMessageBody]() NewCommitTxnMessageBuilderV2 = createNewMessageBuilderV2[*CommitTxnMessageHeader, *CommitTxnMessageBody]() NewRollbackTxnMessageBuilderV2 = createNewMessageBuilderV2[*RollbackTxnMessageHeader, *RollbackTxnMessageBody]() + NewSchemaChangeMessageBuilderV2 = createNewMessageBuilderV2[*SchemaChangeMessageHeader, *SchemaChangeMessageBody]() newTxnMessageBuilderV2 = createNewMessageBuilderV2[*TxnMessageHeader, *TxnMessageBody]() ) diff --git a/pkg/streaming/util/message/message_type.go b/pkg/streaming/util/message/message_type.go index 19dbda3f4e..4dd711437f 100644 --- a/pkg/streaming/util/message/message_type.go +++ b/pkg/streaming/util/message/message_type.go @@ -25,6 +25,7 @@ const ( MessageTypeCommitTxn MessageType = MessageType(messagespb.MessageType_CommitTxn) MessageTypeRollbackTxn MessageType = MessageType(messagespb.MessageType_RollbackTxn) MessageTypeImport MessageType = MessageType(messagespb.MessageType_Import) + MessageTypeSchemaChange MessageType = MessageType(messagespb.MessageType_SchemaChange) ) var messageTypeName = map[MessageType]string{ @@ -44,6 +45,7 @@ var messageTypeName = map[MessageType]string{ MessageTypeCommitTxn: "COMMIT_TXN", MessageTypeRollbackTxn: "ROLLBACK_TXN", MessageTypeImport: "IMPORT", + MessageTypeSchemaChange: "SCHEMA_CHANGE", } // String implements fmt.Stringer interface. diff --git a/pkg/streaming/util/message/specialized_message.go b/pkg/streaming/util/message/specialized_message.go index b8c25a6e96..04665f48a3 100644 --- a/pkg/streaming/util/message/specialized_message.go +++ b/pkg/streaming/util/message/specialized_message.go @@ -29,6 +29,7 @@ type ( RollbackTxnMessageHeader = messagespb.RollbackTxnMessageHeader TxnMessageHeader = messagespb.TxnMessageHeader ImportMessageHeader = messagespb.ImportMessageHeader + SchemaChangeMessageHeader = messagespb.SchemaChangeMessageHeader ) type ( @@ -39,6 +40,7 @@ type ( CommitTxnMessageBody = messagespb.CommitTxnMessageBody RollbackTxnMessageBody = messagespb.RollbackTxnMessageBody TxnMessageBody = messagespb.TxnMessageBody + SchemaChangeMessageBody = messagespb.SchemaChangeMessageBody ) type ( @@ -62,6 +64,7 @@ var messageTypeMap = map[reflect.Type]MessageType{ reflect.TypeOf(&RollbackTxnMessageHeader{}): MessageTypeRollbackTxn, reflect.TypeOf(&TxnMessageHeader{}): MessageTypeTxn, reflect.TypeOf(&ImportMessageHeader{}): MessageTypeImport, + reflect.TypeOf(&SchemaChangeMessageHeader{}): MessageTypeSchemaChange, } // A system preserved message, should not allowed to provide outside of the streaming system. @@ -93,6 +96,7 @@ type ( MutableBeginTxnMessageV2 = specializedMutableMessage[*BeginTxnMessageHeader, *BeginTxnMessageBody] MutableCommitTxnMessageV2 = specializedMutableMessage[*CommitTxnMessageHeader, *CommitTxnMessageBody] MutableRollbackTxnMessageV2 = specializedMutableMessage[*RollbackTxnMessageHeader, *RollbackTxnMessageBody] + MutableSchemaChangeMessageV2 = specializedMutableMessage[*SchemaChangeMessageHeader, *SchemaChangeMessageBody] ImmutableTimeTickMessageV1 = specializedImmutableMessage[*TimeTickMessageHeader, *msgpb.TimeTickMsg] ImmutableInsertMessageV1 = specializedImmutableMessage[*InsertMessageHeader, *msgpb.InsertRequest] @@ -108,6 +112,7 @@ type ( ImmutableBeginTxnMessageV2 = specializedImmutableMessage[*BeginTxnMessageHeader, *BeginTxnMessageBody] ImmutableCommitTxnMessageV2 = specializedImmutableMessage[*CommitTxnMessageHeader, *CommitTxnMessageBody] ImmutableRollbackTxnMessageV2 = specializedImmutableMessage[*RollbackTxnMessageHeader, *RollbackTxnMessageBody] + ImmutableSchemaChangeMessageV2 = specializedImmutableMessage[*SchemaChangeMessageHeader, *SchemaChangeMessageBody] ) // List all as functions for specialized messages. @@ -141,6 +146,7 @@ var ( AsImmutableBeginTxnMessageV2 = asSpecializedImmutableMessage[*BeginTxnMessageHeader, *BeginTxnMessageBody] AsImmutableCommitTxnMessageV2 = asSpecializedImmutableMessage[*CommitTxnMessageHeader, *CommitTxnMessageBody] AsImmutableRollbackTxnMessageV2 = asSpecializedImmutableMessage[*RollbackTxnMessageHeader, *RollbackTxnMessageBody] + AsImmutableCollectionSchemaChangeV2 = asSpecializedImmutableMessage[*SchemaChangeMessageHeader, *SchemaChangeMessageBody] AsImmutableTxnMessage = func(msg ImmutableMessage) ImmutableTxnMessage { underlying, ok := msg.(*immutableTxnMessageImpl) if !ok {